diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 6893cab217ae..3855cf60e6b8 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -44,7 +44,7 @@ This PR has: - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links. - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/licenses.yaml) - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader. -- [ ] added unit tests or modified existing tests to cover new code paths. +- [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met. - [ ] added integration tests. - [ ] been tested in a test Druid cluster. diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml index cd32fb67f732..2da1cb7f2e13 100644 --- a/.idea/inspectionProfiles/Druid.xml +++ b/.idea/inspectionProfiles/Druid.xml @@ -29,6 +29,7 @@ + @@ -387,8 +388,8 @@ - - + + diff --git a/.travis.yml b/.travis.yml index 6dd1adf6629e..9b3c755107c6 100644 --- a/.travis.yml +++ b/.travis.yml @@ -166,7 +166,8 @@ jobs: project_files="$(echo "${all_files}" | grep "${regex}" || [[ $? == 1 ]])"; fi - for f in ${project_files}; do echo $f; done # for debugging - # Check diff code coverage for the maven projects being tested (retry install in case of network error) + # Check diff code coverage for the maven projects being tested (retry install in case of network error). + # Currently, the function coverage check is not reliable, so it is disabled. - > if [ -n "${project_files}" ]; then travis_retry npm install @connectis/diff-test-coverage@1.5.3 @@ -174,9 +175,13 @@ jobs: | node_modules/.bin/diff-test-coverage --coverage "**/target/site/jacoco/jacoco.xml" --type jacoco - --line-coverage 65 - --branch-coverage 65 - --function-coverage 80 + --line-coverage 50 + --branch-coverage 50 + --function-coverage 0 + --log-template "coverage-lines-complete" + --log-template "coverage-files-complete" + --log-template "totals-complete" + --log-template "errors" -- || { printf "\nDiff code coverage check failed. To view coverage report, run 'mvn clean test jacoco:report' and open 'target/site/jacoco/index.html'\n" && false; } fi diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java index 6c0b6d940ed4..f9a920a3d39f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/BoundFilterBenchmark.java @@ -33,6 +33,7 @@ import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.BitmapIndex; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.GenericIndexed; @@ -195,7 +196,7 @@ public CloseableIndexed getDimensionValues(String dimension) } @Override - public boolean hasMultipleValues(final String dimension) + public ColumnCapabilities.Capable hasMultipleValues(final String dimension) { throw new UnsupportedOperationException(); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java index 8dd177f1e831..a845bb1a2233 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DimensionPredicateFilterBenchmark.java @@ -35,6 +35,7 @@ import org.apache.druid.query.filter.DruidLongPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.column.BitmapIndex; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.GenericIndexed; @@ -166,7 +167,7 @@ public CloseableIndexed getDimensionValues(String dimension) } @Override - public boolean hasMultipleValues(final String dimension) + public ColumnCapabilities.Capable hasMultipleValues(final String dimension) { throw new UnsupportedOperationException(); } 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 7d7c8c45f81a..83a13e9a0a67 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -21,8 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import org.apache.druid.benchmark.datagen.BenchmarkColumnSchema; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; import org.apache.druid.benchmark.datagen.SegmentGenerator; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; @@ -42,6 +40,8 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.generator.GeneratorColumnSchema; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; @@ -90,10 +90,10 @@ public void setup() { this.closer = Closer.create(); - final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo( + final GeneratorSchemaInfo schemaInfo = new GeneratorSchemaInfo( ImmutableList.of( - BenchmarkColumnSchema.makeNormal("x", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false), - BenchmarkColumnSchema.makeNormal("y", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false) + GeneratorColumnSchema.makeNormal("x", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false), + GeneratorColumnSchema.makeNormal("y", ValueType.FLOAT, false, 1, 0d, 0d, 10000d, false) ), ImmutableList.of(), Intervals.of("2000/P1D"), 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 6b96d6137357..ea7ac3bdac8f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java @@ -20,8 +20,6 @@ package org.apache.druid.benchmark; import com.google.common.collect.ImmutableList; -import org.apache.druid.benchmark.datagen.BenchmarkColumnSchema; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; import org.apache.druid.benchmark.datagen.SegmentGenerator; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; @@ -39,6 +37,8 @@ import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.generator.GeneratorColumnSchema; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; import org.openjdk.jmh.annotations.Benchmark; @@ -86,9 +86,9 @@ public void setup() { this.closer = Closer.create(); - final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo( + final GeneratorSchemaInfo schemaInfo = new GeneratorSchemaInfo( ImmutableList.of( - BenchmarkColumnSchema.makeEnumerated( + GeneratorColumnSchema.makeEnumerated( "x", ValueType.STRING, false, @@ -97,7 +97,7 @@ public void setup() Arrays.asList("Apple", "Orange", "Xylophone", "Corundum", null), Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) ), - BenchmarkColumnSchema.makeEnumerated( + GeneratorColumnSchema.makeEnumerated( "y", ValueType.STRING, false, 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 d3406a469051..0ee1db3669c7 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -20,8 +20,6 @@ package org.apache.druid.benchmark; import com.google.common.collect.ImmutableList; -import org.apache.druid.benchmark.datagen.BenchmarkColumnSchema; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; import org.apache.druid.benchmark.datagen.SegmentGenerator; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; @@ -41,6 +39,8 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.generator.GeneratorColumnSchema; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -86,9 +86,9 @@ public void setup() { this.closer = Closer.create(); - final BenchmarkSchemaInfo schemaInfo = new BenchmarkSchemaInfo( + final GeneratorSchemaInfo schemaInfo = new GeneratorSchemaInfo( ImmutableList.of( - BenchmarkColumnSchema.makeZipf( + GeneratorColumnSchema.makeZipf( "n", ValueType.LONG, false, @@ -98,7 +98,7 @@ public void setup() 10000, 3d ), - BenchmarkColumnSchema.makeZipf( + GeneratorColumnSchema.makeZipf( "s", ValueType.STRING, false, 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 512f1eb932db..5efe00e5b6aa 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java @@ -22,9 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.jackson.DefaultObjectMapper; @@ -69,6 +66,9 @@ import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.OrFilter; import org.apache.druid.segment.filter.SelectorFilter; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -125,7 +125,7 @@ public class FilterPartitionBenchmark private Filter timeFilterHalf; private Filter timeFilterAll; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private static String JS_FN = "function(str) { return 'super-' + str; }"; private static ExtractionFn JS_EXTRACTION_FN = new JavaScriptExtractionFn(JS_FN, false, JavaScriptConfig.getEnabledInstance()); @@ -153,9 +153,9 @@ public void setup() throws IOException ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde()); - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schema); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java index f60708596a8c..ef4ca7bc1925 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilteredAggregatorBenchmark.java @@ -21,9 +21,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.benchmark.query.QueryBenchmarkUtil; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; @@ -67,6 +64,9 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -125,7 +125,7 @@ public class FilteredAggregatorBenchmark private DimFilter filter; private List inputRows; private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private TimeseriesQuery query; private File tmpDir; @@ -152,9 +152,9 @@ public void setup() throws IOException ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde()); - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schema); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED, schemaInfo.getDataInterval(), @@ -202,7 +202,7 @@ public void setup() throws IOException QueryBenchmarkUtil.NOOP_QUERYWATCHER ); - BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); List queryAggs = new ArrayList<>(); queryAggs.add(filteredMetrics[0]); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java index 1570619b9029..424f2977104c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FloatCompressionBenchmarkFileGenerator.java @@ -20,14 +20,14 @@ package org.apache.druid.benchmark; import com.google.common.collect.ImmutableList; -import org.apache.druid.benchmark.datagen.BenchmarkColumnSchema; -import org.apache.druid.benchmark.datagen.BenchmarkColumnValueGenerator; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ColumnarFloatsSerializer; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.generator.ColumnValueGenerator; +import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import java.io.BufferedReader; @@ -65,7 +65,7 @@ public static void main(String[] args) throws IOException dirPath = args[0]; } - BenchmarkColumnSchema enumeratedSchema = BenchmarkColumnSchema.makeEnumerated( + GeneratorColumnSchema enumeratedSchema = GeneratorColumnSchema.makeEnumerated( "", ValueType.FLOAT, true, @@ -86,7 +86,7 @@ public static void main(String[] args) throws IOException 0.0001 ) ); - BenchmarkColumnSchema zipfLowSchema = BenchmarkColumnSchema.makeZipf( + GeneratorColumnSchema zipfLowSchema = GeneratorColumnSchema.makeZipf( "", ValueType.FLOAT, true, @@ -96,7 +96,7 @@ public static void main(String[] args) throws IOException 1000, 1d ); - BenchmarkColumnSchema zipfHighSchema = BenchmarkColumnSchema.makeZipf( + GeneratorColumnSchema zipfHighSchema = GeneratorColumnSchema.makeZipf( "", ValueType.FLOAT, true, @@ -106,7 +106,7 @@ public static void main(String[] args) throws IOException 1000, 3d ); - BenchmarkColumnSchema sequentialSchema = BenchmarkColumnSchema.makeSequential( + GeneratorColumnSchema sequentialSchema = GeneratorColumnSchema.makeSequential( "", ValueType.FLOAT, true, @@ -115,7 +115,7 @@ public static void main(String[] args) throws IOException 1470187671, 2000000000 ); - BenchmarkColumnSchema uniformSchema = BenchmarkColumnSchema.makeContinuousUniform( + GeneratorColumnSchema uniformSchema = GeneratorColumnSchema.makeContinuousUniform( "", ValueType.FLOAT, true, @@ -125,18 +125,18 @@ public static void main(String[] args) throws IOException 1000 ); - Map generators = new HashMap<>(); - generators.put("enumerate", new BenchmarkColumnValueGenerator(enumeratedSchema, 1)); - generators.put("zipfLow", new BenchmarkColumnValueGenerator(zipfLowSchema, 1)); - generators.put("zipfHigh", new BenchmarkColumnValueGenerator(zipfHighSchema, 1)); - generators.put("sequential", new BenchmarkColumnValueGenerator(sequentialSchema, 1)); - generators.put("uniform", new BenchmarkColumnValueGenerator(uniformSchema, 1)); + Map generators = new HashMap<>(); + generators.put("enumerate", new ColumnValueGenerator(enumeratedSchema, 1)); + generators.put("zipfLow", new ColumnValueGenerator(zipfLowSchema, 1)); + generators.put("zipfHigh", new ColumnValueGenerator(zipfHighSchema, 1)); + generators.put("sequential", new ColumnValueGenerator(sequentialSchema, 1)); + generators.put("uniform", new ColumnValueGenerator(uniformSchema, 1)); File dir = new File(dirPath); dir.mkdir(); // create data files using BenchmarkColunValueGenerator - for (Map.Entry entry : generators.entrySet()) { + for (Map.Entry entry : generators.entrySet()) { final File dataFile = new File(dir, entry.getKey()); dataFile.delete(); try (Writer writer = Files.newBufferedWriter(dataFile.toPath(), StandardCharsets.UTF_8)) { @@ -147,7 +147,7 @@ public static void main(String[] args) throws IOException } // create compressed files using all combinations of CompressionStrategy and FloatEncoding provided - for (Map.Entry entry : generators.entrySet()) { + for (Map.Entry entry : generators.entrySet()) { for (CompressionStrategy compression : COMPRESSIONS) { String name = entry.getKey() + "-" + compression; log.info("%s: ", name); 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 647a787f67b6..d12ff8784f2e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java @@ -23,9 +23,6 @@ import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.google.common.base.Supplier; import com.google.common.base.Suppliers; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.benchmark.query.QueryBenchmarkUtil; import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.DefaultBlockingPool; @@ -70,6 +67,9 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -144,7 +144,7 @@ public class GroupByTypeInterfaceBenchmark private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private GroupByQuery stringQuery; private GroupByQuery longFloatQuery; private GroupByQuery floatQuery; @@ -172,7 +172,7 @@ private void setupQueries() { // queries for the basic schema Map basicQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); { // basic.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -277,13 +277,13 @@ public void setup() throws IOException String schemaName = "basic"; - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); stringQuery = SCHEMA_QUERY_MAP.get(schemaName).get("string"); longFloatQuery = SCHEMA_QUERY_MAP.get(schemaName).get("longFloat"); longQuery = SCHEMA_QUERY_MAP.get(schemaName).get("long"); floatQuery = SCHEMA_QUERY_MAP.get(schemaName).get("float"); - final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator( + final DataGenerator dataGenerator = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED + 1, schemaInfo.getDataInterval(), 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 9098953fbe97..132a3215927b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java @@ -38,6 +38,7 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnConfig; @@ -48,8 +49,8 @@ import org.apache.druid.segment.join.JoinTestHelper; import org.apache.druid.segment.join.JoinType; import org.apache.druid.segment.join.JoinableClause; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -139,19 +140,20 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisLookupStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClausesLookupStringKey, - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + JoinFilterPreAnalysisGroup preAnalysisGroupLookupStringKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); + hashJoinLookupStringKeySegment = new HashJoinSegment( - baseSegment, + ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesLookupStringKey, - preAnalysisLookupStringKey + preAnalysisGroupLookupStringKey ); List joinableClausesLookupLongKey = ImmutableList.of( @@ -166,19 +168,20 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisLookupLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClausesLookupLongKey, - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + + JoinFilterPreAnalysisGroup preAnalysisGroupLookupLongKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); hashJoinLookupLongKeySegment = new HashJoinSegment( - baseSegment, + ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesLookupLongKey, - preAnalysisLookupLongKey + preAnalysisGroupLookupLongKey ); List joinableClausesIndexedTableStringKey = ImmutableList.of( @@ -193,19 +196,20 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisIndexedTableStringKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClausesIndexedTableStringKey, - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + + JoinFilterPreAnalysisGroup preAnalysisGroupIndexedStringKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); hashJoinIndexedTableStringKeySegment = new HashJoinSegment( - baseSegment, + ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesIndexedTableStringKey, - preAnalysisIndexedTableStringKey + preAnalysisGroupIndexedStringKey ); List joinableClausesIndexedTableLonggKey = ImmutableList.of( @@ -220,19 +224,19 @@ public void setup() throws IOException ) ) ); - JoinFilterPreAnalysis preAnalysisIndexedTableLongKey = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClausesIndexedTableLonggKey, - VirtualColumns.EMPTY, - null, - false, - false, - false, - 0 + JoinFilterPreAnalysisGroup preAnalysisGroupIndexedLongKey = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + false, + false, + 0 + ), + true ); hashJoinIndexedTableLongKeySegment = new HashJoinSegment( - baseSegment, + ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClausesIndexedTableLonggKey, - preAnalysisIndexedTableLongKey + preAnalysisGroupIndexedLongKey ); final Map countryCodeToNameMap = JoinTestHelper.createCountryIsoCodeToNameLookup().getMap(); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java index 83551146eb24..5c4a0f35a60e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/LikeFilterBenchmark.java @@ -35,6 +35,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.BitmapIndex; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.GenericIndexed; @@ -166,7 +167,7 @@ public CloseableIndexed getDimensionValues(String dimension) } @Override - public boolean hasMultipleValues(final String dimension) + public ColumnCapabilities.Capable hasMultipleValues(final String dimension) { throw new UnsupportedOperationException(); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/LongCompressionBenchmarkFileGenerator.java b/benchmarks/src/test/java/org/apache/druid/benchmark/LongCompressionBenchmarkFileGenerator.java index ead3dd0e5cc7..b9bca954de45 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/LongCompressionBenchmarkFileGenerator.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/LongCompressionBenchmarkFileGenerator.java @@ -20,14 +20,14 @@ package org.apache.druid.benchmark; import com.google.common.collect.ImmutableList; -import org.apache.druid.benchmark.datagen.BenchmarkColumnSchema; -import org.apache.druid.benchmark.datagen.BenchmarkColumnValueGenerator; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ColumnarLongsSerializer; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.generator.ColumnValueGenerator; +import org.apache.druid.segment.generator.GeneratorColumnSchema; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMedium; import java.io.BufferedReader; @@ -66,7 +66,7 @@ public static void main(String[] args) throws IOException dirPath = args[0]; } - BenchmarkColumnSchema enumeratedSchema = BenchmarkColumnSchema.makeEnumerated( + GeneratorColumnSchema enumeratedSchema = GeneratorColumnSchema.makeEnumerated( "", ValueType.LONG, true, @@ -87,8 +87,8 @@ public static void main(String[] args) throws IOException 0.0001 ) ); - BenchmarkColumnSchema zipfLowSchema = BenchmarkColumnSchema.makeZipf("", ValueType.LONG, true, 1, 0d, -1, 1000, 1d); - BenchmarkColumnSchema zipfHighSchema = BenchmarkColumnSchema.makeZipf( + GeneratorColumnSchema zipfLowSchema = GeneratorColumnSchema.makeZipf("", ValueType.LONG, true, 1, 0d, -1, 1000, 1d); + GeneratorColumnSchema zipfHighSchema = GeneratorColumnSchema.makeZipf( "", ValueType.LONG, true, @@ -98,7 +98,7 @@ public static void main(String[] args) throws IOException 1000, 3d ); - BenchmarkColumnSchema sequentialSchema = BenchmarkColumnSchema.makeSequential( + GeneratorColumnSchema sequentialSchema = GeneratorColumnSchema.makeSequential( "", ValueType.LONG, true, @@ -107,7 +107,7 @@ public static void main(String[] args) throws IOException 1470187671, 2000000000 ); - BenchmarkColumnSchema uniformSchema = BenchmarkColumnSchema.makeDiscreteUniform( + GeneratorColumnSchema uniformSchema = GeneratorColumnSchema.makeDiscreteUniform( "", ValueType.LONG, true, @@ -117,18 +117,18 @@ public static void main(String[] args) throws IOException 1000 ); - Map generators = new HashMap<>(); - generators.put("enumerate", new BenchmarkColumnValueGenerator(enumeratedSchema, 1)); - generators.put("zipfLow", new BenchmarkColumnValueGenerator(zipfLowSchema, 1)); - generators.put("zipfHigh", new BenchmarkColumnValueGenerator(zipfHighSchema, 1)); - generators.put("sequential", new BenchmarkColumnValueGenerator(sequentialSchema, 1)); - generators.put("uniform", new BenchmarkColumnValueGenerator(uniformSchema, 1)); + Map generators = new HashMap<>(); + generators.put("enumerate", new ColumnValueGenerator(enumeratedSchema, 1)); + generators.put("zipfLow", new ColumnValueGenerator(zipfLowSchema, 1)); + generators.put("zipfHigh", new ColumnValueGenerator(zipfHighSchema, 1)); + generators.put("sequential", new ColumnValueGenerator(sequentialSchema, 1)); + generators.put("uniform", new ColumnValueGenerator(uniformSchema, 1)); File dir = new File(dirPath); dir.mkdir(); // create data files using BenchmarkColunValueGenerator - for (Map.Entry entry : generators.entrySet()) { + for (Map.Entry entry : generators.entrySet()) { final File dataFile = new File(dir, entry.getKey()); dataFile.delete(); try (Writer writer = Files.newBufferedWriter(dataFile.toPath(), StandardCharsets.UTF_8)) { @@ -139,7 +139,7 @@ public static void main(String[] args) throws IOException } // create compressed files using all combinations of CompressionStrategy and LongEncoding provided - for (Map.Entry entry : generators.entrySet()) { + for (Map.Entry entry : generators.entrySet()) { for (CompressionStrategy compression : COMPRESSIONS) { for (CompressionFactory.LongEncodingStrategy encoding : ENCODINGS) { String name = entry.getKey() + "-" + compression + "-" + encoding; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java index 69f48664555f..bf7733db180e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/TopNTypeInterfaceBenchmark.java @@ -20,9 +20,6 @@ package org.apache.druid.benchmark; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.benchmark.query.QueryBenchmarkUtil; import org.apache.druid.collections.StupidPool; import org.apache.druid.common.config.NullHandling; @@ -66,6 +63,9 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -125,7 +125,7 @@ public class TopNTypeInterfaceBenchmark private List qIndexes; private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private TopNQueryBuilder queryBuilder; private TopNQuery stringQuery; private TopNQuery longQuery; @@ -153,7 +153,7 @@ private void setupQueries() { // queries for the basic schema Map basicQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); { // basic.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -165,7 +165,7 @@ private void setupQueries() queryAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "minFloatZipf")); queryAggs.add(new HyperUniquesAggregatorFactory("hyperUniquesMet", "hyper")); - // Use an IdentityExtractionFn to force usage of DimExtractionTopNAlgorithm + // Use an IdentityExtractionFn to force usage of HeapBasedTopNAlgorithm TopNQueryBuilder queryBuilderString = new TopNQueryBuilder() .dataSource("blah") .granularity(Granularities.ALL) @@ -174,7 +174,7 @@ private void setupQueries() .intervals(intervalSpec) .aggregators(queryAggs); - // DimExtractionTopNAlgorithm is always used for numeric columns + // HeapBasedTopNAlgorithm is always used for numeric columns TopNQueryBuilder queryBuilderLong = new TopNQueryBuilder() .dataSource("blah") .granularity(Granularities.ALL) @@ -241,7 +241,7 @@ public void setup() throws IOException setupQueries(); - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); queryBuilder = SCHEMA_QUERY_MAP.get("basic").get("string"); queryBuilder.threshold(threshold); stringQuery = queryBuilder.build(); @@ -258,7 +258,7 @@ public void setup() throws IOException for (int i = 0; i < numSegments; i++) { log.info("Generating rows for segment " + i); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED + i, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkSchemas.java b/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkSchemas.java deleted file mode 100644 index 142843753fdc..000000000000 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkSchemas.java +++ /dev/null @@ -1,297 +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.benchmark.datagen; - -import com.google.common.collect.ImmutableList; -import org.apache.druid.java.util.common.Intervals; -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.aggregation.DoubleMinAggregatorFactory; -import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; -import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; -import org.apache.druid.segment.column.ValueType; -import org.joda.time.Interval; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; - -public class BenchmarkSchemas -{ - public static final Map SCHEMA_MAP = new LinkedHashMap<>(); - - static { // basic schema - List basicSchemaColumns = ImmutableList.of( - // dims - BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.STRING, false, 1, null, 0, 1000), - BenchmarkColumnSchema.makeZipf("dimZipf", ValueType.STRING, false, 1, null, 1, 101, 1.0), - BenchmarkColumnSchema.makeDiscreteUniform("dimUniform", ValueType.STRING, false, 1, null, 1, 100000), - BenchmarkColumnSchema.makeSequential("dimSequentialHalfNull", ValueType.STRING, false, 1, 0.5, 0, 1000), - BenchmarkColumnSchema.makeEnumerated( - "dimMultivalEnumerated", - ValueType.STRING, - false, - 4, - null, - Arrays.asList("Hello", "World", "Foo", "Bar", "Baz"), - Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) - ), - BenchmarkColumnSchema.makeEnumerated( - "dimMultivalEnumerated2", - ValueType.STRING, - false, - 3, - null, - Arrays.asList("Apple", "Orange", "Xylophone", "Corundum", null), - Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) - ), - BenchmarkColumnSchema.makeSequential("dimMultivalSequentialWithNulls", ValueType.STRING, false, 8, 0.15, 1, 11), - BenchmarkColumnSchema.makeSequential("dimHyperUnique", ValueType.STRING, false, 1, null, 0, 100000), - BenchmarkColumnSchema.makeSequential("dimNull", ValueType.STRING, false, 1, 1.0, 0, 1), - - // metrics - BenchmarkColumnSchema.makeSequential("metLongSequential", ValueType.LONG, true, 1, null, 0, 10000), - BenchmarkColumnSchema.makeDiscreteUniform("metLongUniform", ValueType.LONG, true, 1, null, 0, 500), - BenchmarkColumnSchema.makeNormal("metFloatNormal", ValueType.FLOAT, true, 1, null, 5000.0, 1.0, true), - BenchmarkColumnSchema.makeZipf("metFloatZipf", ValueType.FLOAT, true, 1, null, 0, 1000, 1.0) - ); - - List basicSchemaIngestAggs = new ArrayList<>(); - basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); - basicSchemaIngestAggs.add(new LongSumAggregatorFactory("sumLongSequential", "metLongSequential")); - basicSchemaIngestAggs.add(new LongMaxAggregatorFactory("maxLongUniform", "metLongUniform")); - basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("sumFloatNormal", "metFloatNormal")); - basicSchemaIngestAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf")); - basicSchemaIngestAggs.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique")); - - List basicSchemaIngestAggsExpression = new ArrayList<>(); - basicSchemaIngestAggsExpression.add(new CountAggregatorFactory("rows")); - basicSchemaIngestAggsExpression.add(new LongSumAggregatorFactory("sumLongSequential", null, "if(sumLongSequential>0 && dimSequential>100 || dimSequential<10 || metLongSequential>3000,sumLongSequential,0)", ExprMacroTable.nil())); - basicSchemaIngestAggsExpression.add(new LongMaxAggregatorFactory("maxLongUniform", "metLongUniform")); - basicSchemaIngestAggsExpression.add(new DoubleSumAggregatorFactory("sumFloatNormal", null, "if(sumFloatNormal>0 && dimSequential>100 || dimSequential<10 || metLongSequential>3000,sumFloatNormal,0)", ExprMacroTable.nil())); - basicSchemaIngestAggsExpression.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf")); - basicSchemaIngestAggsExpression.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique")); - - Interval basicSchemaDataInterval = Intervals.of("2000-01-01/P1D"); - - BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( - basicSchemaColumns, - basicSchemaIngestAggs, - basicSchemaDataInterval, - true - ); - - BenchmarkSchemaInfo basicSchemaExpression = new BenchmarkSchemaInfo( - basicSchemaColumns, - basicSchemaIngestAggsExpression, - basicSchemaDataInterval, - true - ); - - SCHEMA_MAP.put("basic", basicSchema); - SCHEMA_MAP.put("expression", basicSchemaExpression); - } - - static { // simple single string column and count agg schema, no rollup - List basicSchemaColumns = ImmutableList.of( - // dims - BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.STRING, false, 1, null, 0, 1000000) - ); - - List basicSchemaIngestAggs = new ArrayList<>(); - basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); - - Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); - - BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( - basicSchemaColumns, - basicSchemaIngestAggs, - basicSchemaDataInterval, - false - ); - SCHEMA_MAP.put("simple", basicSchema); - } - - static { // simple single long column and count agg schema, no rollup - List basicSchemaColumns = ImmutableList.of( - // dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported - BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.LONG, true, 1, null, 0, 1000000) - ); - - List basicSchemaIngestAggs = new ArrayList<>(); - basicSchemaIngestAggs.add(new LongSumAggregatorFactory("dimSequential", "dimSequential")); - basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); - - Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); - - BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( - basicSchemaColumns, - basicSchemaIngestAggs, - basicSchemaDataInterval, - false - ); - SCHEMA_MAP.put("simpleLong", basicSchema); - } - - static { // simple single float column and count agg schema, no rollup - List basicSchemaColumns = ImmutableList.of( - // dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported - BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.FLOAT, true, 1, null, 0, 1000000) - ); - - List basicSchemaIngestAggs = new ArrayList<>(); - basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("dimSequential", "dimSequential")); - basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); - - Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); - - BenchmarkSchemaInfo basicSchema = new BenchmarkSchemaInfo( - basicSchemaColumns, - basicSchemaIngestAggs, - basicSchemaDataInterval, - false - ); - SCHEMA_MAP.put("simpleFloat", basicSchema); - } - - static { // schema with high opportunity for rollup - List rolloColumns = ImmutableList.of( - // dims - BenchmarkColumnSchema.makeEnumerated( - "dimEnumerated", - ValueType.STRING, - false, - 1, - null, - Arrays.asList("Hello", "World", "Foo", "Bar", "Baz"), - Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) - ), - BenchmarkColumnSchema.makeEnumerated( - "dimEnumerated2", - ValueType.STRING, - false, - 1, - null, - Arrays.asList("Apple", "Orange", "Xylophone", "Corundum", null), - Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) - ), - BenchmarkColumnSchema.makeZipf("dimZipf", ValueType.STRING, false, 1, null, 1, 100, 2.0), - BenchmarkColumnSchema.makeDiscreteUniform("dimUniform", ValueType.STRING, false, 1, null, 1, 100), - - // metrics - BenchmarkColumnSchema.makeZipf("metLongZipf", ValueType.LONG, true, 1, null, 0, 10000, 2.0), - BenchmarkColumnSchema.makeDiscreteUniform("metLongUniform", ValueType.LONG, true, 1, null, 0, 500), - BenchmarkColumnSchema.makeNormal("metFloatNormal", ValueType.FLOAT, true, 1, null, 5000.0, 1.0, true), - BenchmarkColumnSchema.makeZipf("metFloatZipf", ValueType.FLOAT, true, 1, null, 0, 1000, 1.5) - ); - - List rolloSchemaIngestAggs = new ArrayList<>(); - rolloSchemaIngestAggs.add(new CountAggregatorFactory("rows")); - rolloSchemaIngestAggs.add(new LongSumAggregatorFactory("sumLongSequential", "metLongSequential")); - rolloSchemaIngestAggs.add(new LongMaxAggregatorFactory("maxLongUniform", "metLongUniform")); - rolloSchemaIngestAggs.add(new DoubleSumAggregatorFactory("sumFloatNormal", "metFloatNormal")); - rolloSchemaIngestAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf")); - rolloSchemaIngestAggs.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique")); - - Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); - - BenchmarkSchemaInfo rolloSchema = new BenchmarkSchemaInfo( - rolloColumns, - rolloSchemaIngestAggs, - basicSchemaDataInterval, - true - ); - SCHEMA_MAP.put("rollo", rolloSchema); - } - - static { // simple schema with null valued rows, no aggs on numeric columns - List nullsSchemaColumns = ImmutableList.of( - // string dims with nulls - BenchmarkColumnSchema.makeZipf("stringZipf", ValueType.STRING, false, 1, 0.8, 1, 101, 1.5), - BenchmarkColumnSchema.makeDiscreteUniform("stringUniform", ValueType.STRING, false, 1, 0.3, 1, 100000), - BenchmarkColumnSchema.makeSequential("stringSequential", ValueType.STRING, false, 1, 0.5, 0, 1000), - - // numeric dims with nulls - BenchmarkColumnSchema.makeSequential("longSequential", ValueType.LONG, false, 1, 0.45, 0, 10000), - BenchmarkColumnSchema.makeDiscreteUniform("longUniform", ValueType.LONG, false, 1, 0.25, 0, 500), - BenchmarkColumnSchema.makeZipf("doubleZipf", ValueType.DOUBLE, false, 1, 0.1, 0, 1000, 2.0), - BenchmarkColumnSchema.makeZipf("floatZipf", ValueType.FLOAT, false, 1, 0.1, 0, 1000, 2.0) - ); - - List simpleNullsSchemaIngestAggs = new ArrayList<>(); - simpleNullsSchemaIngestAggs.add(new CountAggregatorFactory("rows")); - - Interval nullsSchemaDataInterval = Intervals.of("2000-01-01/P1D"); - - BenchmarkSchemaInfo nullsSchema = new BenchmarkSchemaInfo( - nullsSchemaColumns, - simpleNullsSchemaIngestAggs, - nullsSchemaDataInterval, - false - ); - - SCHEMA_MAP.put("nulls", nullsSchema); - } - - static { // simple schema with null valued rows, no aggs on numeric columns - List nullsSchemaColumns = ImmutableList.of( - // string dims - BenchmarkColumnSchema.makeZipf("stringZipf", ValueType.STRING, false, 1, null, 1, 101, 1.5), - BenchmarkColumnSchema.makeDiscreteUniform("stringUniform", ValueType.STRING, false, 1, null, 1, 100000), - BenchmarkColumnSchema.makeSequential("stringSequential", ValueType.STRING, false, 1, null, 0, 1000), - - // numeric dims - BenchmarkColumnSchema.makeSequential("longSequential", ValueType.LONG, false, 1, null, 0, 10000), - BenchmarkColumnSchema.makeDiscreteUniform("longUniform", ValueType.LONG, false, 1, null, 0, 500), - BenchmarkColumnSchema.makeZipf("doubleZipf", ValueType.DOUBLE, false, 1, null, 0, 1000, 2.0), - BenchmarkColumnSchema.makeZipf("floatZipf", ValueType.FLOAT, false, 1, null, 0, 1000, 2.0), - - // string dims with nulls - BenchmarkColumnSchema.makeZipf("stringZipfWithNulls", ValueType.STRING, false, 1, 0.8, 1, 101, 1.5), - BenchmarkColumnSchema.makeDiscreteUniform("stringUniformWithNulls", ValueType.STRING, false, 1, 0.3, 1, 100000), - BenchmarkColumnSchema.makeSequential("stringSequentialWithNulls", ValueType.STRING, false, 1, 0.5, 0, 1000), - - // numeric dims with nulls - BenchmarkColumnSchema.makeSequential("longSequentialWithNulls", ValueType.LONG, false, 1, 0.45, 0, 10000), - BenchmarkColumnSchema.makeDiscreteUniform("longUniformWithNulls", ValueType.LONG, false, 1, 0.25, 0, 500), - BenchmarkColumnSchema.makeZipf("doubleZipfWithNulls", ValueType.DOUBLE, false, 1, 0.1, 0, 1000, 2.0), - BenchmarkColumnSchema.makeZipf("floatZipfWithNulls", ValueType.FLOAT, false, 1, 0.1, 0, 1000, 2.0) - ); - - List simpleNullsSchemaIngestAggs = new ArrayList<>(); - simpleNullsSchemaIngestAggs.add(new CountAggregatorFactory("rows")); - - Interval nullsSchemaDataInterval = Intervals.of("2000-01-01/P1D"); - - BenchmarkSchemaInfo nullsSchema = new BenchmarkSchemaInfo( - nullsSchemaColumns, - simpleNullsSchemaIngestAggs, - nullsSchemaDataInterval, - false - ); - - SCHEMA_MAP.put("nulls-and-non-nulls", nullsSchema); - } -} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java b/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java index 95a42d5ffd88..f6d018d9bb82 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/SegmentGenerator.java @@ -35,6 +35,8 @@ import org.apache.druid.segment.QueryableIndexIndexableAdapter; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -100,7 +102,7 @@ public File getCacheDir() public QueryableIndex generate( final DataSegment dataSegment, - final BenchmarkSchemaInfo schemaInfo, + final GeneratorSchemaInfo schemaInfo, final Granularity granularity, final int numRows ) @@ -131,7 +133,7 @@ public QueryableIndex generate( log.info("Writing segment with hash[%s] to directory[%s].", dataHash, outDir); - final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator( + final DataGenerator dataGenerator = new DataGenerator( schemaInfo.getColumnSchemas(), dataSegment.getId().hashCode(), /* Use segment identifier hashCode as seed */ schemaInfo.getDataInterval(), 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 f7d1e91b5236..1a9ac01cf412 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 @@ -19,9 +19,6 @@ package org.apache.druid.benchmark.indexing; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.granularity.Granularities; @@ -43,6 +40,9 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; @@ -91,7 +91,7 @@ public class IncrementalIndexReadBenchmark private IncrementalIndex incIndex; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; @Setup public void setup() throws IOException @@ -100,9 +100,9 @@ public void setup() throws IOException ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde()); - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schema); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java index 19862f27c948..02c6efd89569 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexIngestionBenchmark.java @@ -19,13 +19,13 @@ package org.apache.druid.benchmark.indexing; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesSerde; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.serde.ComplexMetrics; @@ -70,7 +70,7 @@ public class IndexIngestionBenchmark private IncrementalIndex incIndex; private ArrayList rows; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; @Setup public void setup() @@ -78,9 +78,9 @@ public void setup() ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde()); rows = new ArrayList(); - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schema); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java index 6b3b902c3ad4..80e964362e44 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexMergeBenchmark.java @@ -21,9 +21,6 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.jackson.DefaultObjectMapper; @@ -35,6 +32,9 @@ import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.serde.ComplexMetrics; @@ -96,7 +96,7 @@ public class IndexMergeBenchmark } private List indexesToMerge; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private File tmpDir; private IndexMergerV9 indexMergerV9; @@ -121,10 +121,10 @@ public void setup() throws IOException indexesToMerge = new ArrayList<>(); - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schema); for (int i = 0; i < numSegments; i++) { - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED + i, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java index d826fe8fdc73..5b1a0ca6d6f4 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IndexPersistBenchmark.java @@ -20,9 +20,6 @@ package org.apache.druid.benchmark.indexing; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.jackson.DefaultObjectMapper; @@ -32,6 +29,9 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexMergerV9; import org.apache.druid.segment.IndexSpec; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.serde.ComplexMetrics; @@ -92,7 +92,7 @@ public class IndexPersistBenchmark private IncrementalIndex incIndex; private ArrayList rows; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; @Setup public void setup() @@ -102,7 +102,7 @@ public void setup() ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde()); rows = new ArrayList(); - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schema); int valuesPerTimestamp = 1; switch (rollupOpportunity) { @@ -115,7 +115,7 @@ public void setup() } - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED, schemaInfo.getDataInterval().getStartMillis(), 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 94b560e8f4df..daa614ff4ddd 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 @@ -27,8 +27,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Ordering; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.benchmark.datagen.SegmentGenerator; import org.apache.druid.client.CachingClusteredClient; import org.apache.druid.client.DruidServer; @@ -104,6 +102,8 @@ import org.apache.druid.query.topn.TopNResultValue; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; @@ -180,7 +180,7 @@ public class CachingClusteredClientBenchmark private final Closer closer = Closer.create(); - private final BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + private final GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); private final QuerySegmentSpec basicSchemaIntervalSpec = new MultipleIntervalSegmentSpec( Collections.singletonList(basicSchema.getDataInterval()) ); @@ -204,7 +204,7 @@ public void setup() parallelCombine = parallelism > 0; - BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); Map queryableIndexes = Maps.newHashMapWithExpectedSize(numServers); 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 02107f09af48..57214154cd2c 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 @@ -26,9 +26,6 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.collections.BlockingPool; import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.collections.NonBlockingPool; @@ -84,6 +81,9 @@ import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.serde.ComplexMetrics; @@ -161,7 +161,7 @@ public class GroupByBenchmark private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private GroupByQuery query; private ExecutorService executorService; @@ -192,7 +192,7 @@ private void setupQueries() { // queries for the basic schema Map basicQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); { // basic.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -324,7 +324,7 @@ private void setupQueries() // simple one column schema, for testing performance difference between querying on numeric values as Strings and // directly as longs Map simpleQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo simpleSchema = BenchmarkSchemas.SCHEMA_MAP.get("simple"); + GeneratorSchemaInfo simpleSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("simple"); { // simple.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleSchema.getDataInterval())); @@ -351,7 +351,7 @@ private void setupQueries() Map simpleLongQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo simpleLongSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleLong"); + GeneratorSchemaInfo simpleLongSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("simpleLong"); { // simpleLong.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleLongSchema.getDataInterval())); List queryAggs = new ArrayList<>(); @@ -377,7 +377,7 @@ private void setupQueries() Map simpleFloatQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo simpleFloatSchema = BenchmarkSchemas.SCHEMA_MAP.get("simpleFloat"); + GeneratorSchemaInfo simpleFloatSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("simpleFloat"); { // simpleFloat.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(simpleFloatSchema.getDataInterval())); List queryAggs = new ArrayList<>(); @@ -402,7 +402,7 @@ private void setupQueries() // simple one column schema, for testing performance difference between querying on numeric values as Strings and // directly as longs Map nullQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo nullSchema = BenchmarkSchemas.SCHEMA_MAP.get("nulls"); + GeneratorSchemaInfo nullSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("nulls"); { // simple-null QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(nullSchema.getDataInterval())); @@ -443,10 +443,10 @@ public void setup() throws IOException String schemaName = schemaQuery[0]; String queryName = schemaQuery[1]; - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); query = SCHEMA_QUERY_MAP.get(schemaName).get(queryName); - final BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator( + final DataGenerator dataGenerator = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED + 1, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java index a2b0e98fb980..410b13e4661c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/ScanBenchmark.java @@ -22,9 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.Row; @@ -67,6 +64,9 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -133,7 +133,7 @@ public class ScanBenchmark private List qIndexes; private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private Druids.ScanQueryBuilder queryBuilder; private ScanQuery query; private File tmpDir; @@ -155,7 +155,7 @@ private void setupQueries() { // queries for the basic schema final Map basicQueries = new LinkedHashMap<>(); - final BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + final GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); final List queryTypes = ImmutableList.of("A", "B", "C", "D"); for (final String eachType : queryTypes) { @@ -165,7 +165,7 @@ private void setupQueries() SCHEMA_QUERY_MAP.put("basic", basicQueries); } - private static Druids.ScanQueryBuilder makeQuery(final String name, final BenchmarkSchemaInfo basicSchema) + private static Druids.ScanQueryBuilder makeQuery(final String name, final GeneratorSchemaInfo basicSchema) { switch (name) { case "A": @@ -182,7 +182,7 @@ private static Druids.ScanQueryBuilder makeQuery(final String name, final Benchm } /* Just get everything */ - private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) + private static Druids.ScanQueryBuilder basicA(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -193,7 +193,7 @@ private static Druids.ScanQueryBuilder basicA(final BenchmarkSchemaInfo basicSch .order(ordering); } - private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) + private static Druids.ScanQueryBuilder basicB(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -214,7 +214,7 @@ private static Druids.ScanQueryBuilder basicB(final BenchmarkSchemaInfo basicSch .order(ordering); } - private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) + private static Druids.ScanQueryBuilder basicC(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -227,7 +227,7 @@ private static Druids.ScanQueryBuilder basicC(final BenchmarkSchemaInfo basicSch .order(ordering); } - private static Druids.ScanQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) + private static Druids.ScanQueryBuilder basicD(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( Collections.singletonList(basicSchema.getDataInterval()) @@ -257,7 +257,7 @@ public void setup() throws IOException String schemaName = schemaQuery[0]; String queryName = schemaQuery[1]; - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); queryBuilder = SCHEMA_QUERY_MAP.get(schemaName).get(queryName); queryBuilder.limit(limit); query = queryBuilder.build(); @@ -265,7 +265,7 @@ public void setup() throws IOException incIndexes = new ArrayList<>(); for (int i = 0; i < numSegments; i++) { log.info("Generating rows for segment " + i); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), System.currentTimeMillis(), schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java index 06320c331101..d8ae55adc412 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SearchBenchmark.java @@ -24,9 +24,6 @@ import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.Row; @@ -74,6 +71,9 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -133,7 +133,7 @@ public class SearchBenchmark private List qIndexes; private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private Druids.SearchQueryBuilder queryBuilder; private SearchQuery query; private File tmpDir; @@ -162,7 +162,7 @@ private void setupQueries() { // queries for the basic schema final Map basicQueries = new LinkedHashMap<>(); - final BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + final GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); final List queryTypes = ImmutableList.of("A", "B", "C", "D"); for (final String eachType : queryTypes) { @@ -172,7 +172,7 @@ private void setupQueries() SCHEMA_QUERY_MAP.put("basic", basicQueries); } - private static SearchQueryBuilder makeQuery(final String name, final BenchmarkSchemaInfo basicSchema) + private static SearchQueryBuilder makeQuery(final String name, final GeneratorSchemaInfo basicSchema) { switch (name) { case "A": @@ -188,7 +188,7 @@ private static SearchQueryBuilder makeQuery(final String name, final BenchmarkSc } } - private static SearchQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) + private static SearchQueryBuilder basicA(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -199,7 +199,7 @@ private static SearchQueryBuilder basicA(final BenchmarkSchemaInfo basicSchema) .query("123"); } - private static SearchQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) + private static SearchQueryBuilder basicB(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -230,7 +230,7 @@ private static SearchQueryBuilder basicB(final BenchmarkSchemaInfo basicSchema) .filters(new AndDimFilter(dimFilters)); } - private static SearchQueryBuilder basicC(final BenchmarkSchemaInfo basicSchema) + private static SearchQueryBuilder basicC(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -284,7 +284,7 @@ public ExtractionType getExtractionType() .filters(new AndDimFilter(dimFilters)); } - private static SearchQueryBuilder basicD(final BenchmarkSchemaInfo basicSchema) + private static SearchQueryBuilder basicD(final GeneratorSchemaInfo basicSchema) { final QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec( Collections.singletonList(basicSchema.getDataInterval()) @@ -327,7 +327,7 @@ public void setup() throws IOException String schemaName = schemaQuery[0]; String queryName = schemaQuery[1]; - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); queryBuilder = SCHEMA_QUERY_MAP.get(schemaName).get(queryName); queryBuilder.limit(limit); query = queryBuilder.build(); @@ -335,7 +335,7 @@ public void setup() throws IOException incIndexes = new ArrayList<>(); for (int i = 0; i < numSegments; i++) { log.info("Generating rows for segment " + i); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), System.currentTimeMillis(), schemaInfo.getDataInterval(), 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 6c2de09cce55..bd63a1a20e9d 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 @@ -22,8 +22,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.calcite.schema.SchemaPlus; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.benchmark.datagen.SegmentGenerator; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.granularity.Granularities; @@ -32,6 +30,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthenticationResult; @@ -385,7 +385,7 @@ public class SqlBenchmark @Setup(Level.Trial) public void setup() { - final BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); final DataSegment dataSegment = DataSegment.builder() .dataSource("foo") diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index dc75b6d47d1f..2e89c4dda48e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -21,8 +21,6 @@ import com.google.common.collect.ImmutableList; import org.apache.calcite.schema.SchemaPlus; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.benchmark.datagen.SegmentGenerator; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Intervals; @@ -38,6 +36,8 @@ import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.security.AuthTestUtils; import org.apache.druid.server.security.AuthenticationResult; @@ -96,7 +96,7 @@ public void setup() { this.closer = Closer.create(); - final BenchmarkSchemaInfo schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + final GeneratorSchemaInfo schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); final DataSegment dataSegment = DataSegment.builder() .dataSource("foo") diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java index bf72eea3774e..931af6cf5f7b 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TimeseriesBenchmark.java @@ -20,9 +20,6 @@ package org.apache.druid.benchmark.query; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.jackson.DefaultObjectMapper; @@ -68,6 +65,9 @@ import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -126,7 +126,7 @@ public class TimeseriesBenchmark private File tmpDir; private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private TimeseriesQuery query; private ExecutorService executorService; @@ -153,7 +153,7 @@ private void setupQueries() { // queries for the basic schema Map basicQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); { // basic.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -253,13 +253,13 @@ public void setup() throws IOException String schemaName = schemaQuery[0]; String queryName = schemaQuery[1]; - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); query = SCHEMA_QUERY_MAP.get(schemaName).get(queryName); incIndexes = new ArrayList<>(); for (int i = 0; i < numSegments; i++) { log.info("Generating rows for segment " + i); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED + i, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java index c9ccfd7828e2..bbafeff1507f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/TopNBenchmark.java @@ -20,9 +20,6 @@ package org.apache.druid.benchmark.query; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.collections.StupidPool; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; @@ -65,6 +62,9 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -125,7 +125,7 @@ public class TopNBenchmark private List qIndexes; private QueryRunnerFactory factory; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private TopNQueryBuilder queryBuilder; private TopNQuery query; private File tmpDir; @@ -154,7 +154,7 @@ private void setupQueries() { // queries for the basic schema Map basicQueries = new LinkedHashMap<>(); - BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); { // basic.A QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -228,7 +228,7 @@ public void setup() throws IOException String schemaName = schemaQuery[0]; String queryName = schemaQuery[1]; - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); queryBuilder = SCHEMA_QUERY_MAP.get(schemaName).get(queryName); queryBuilder.threshold(threshold); query = queryBuilder.build(); @@ -237,7 +237,7 @@ public void setup() throws IOException for (int i = 0; i < numSegments; i++) { log.info("Generating rows for segment " + i); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED + i, schemaInfo.getDataInterval(), diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java index f87773d36f1e..3272ecd2623d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/timecompare/TimeCompareBenchmark.java @@ -21,9 +21,6 @@ import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; -import org.apache.druid.benchmark.datagen.BenchmarkSchemaInfo; -import org.apache.druid.benchmark.datagen.BenchmarkSchemas; import org.apache.druid.benchmark.query.QueryBenchmarkUtil; import org.apache.druid.collections.StupidPool; import org.apache.druid.common.config.NullHandling; @@ -72,6 +69,9 @@ import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.serde.ComplexMetrics; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; @@ -143,7 +143,7 @@ public class TimeCompareBenchmark private Query timeseriesQuery; private QueryRunner timeseriesRunner; - private BenchmarkSchemaInfo schemaInfo; + private GeneratorSchemaInfo schemaInfo; private File tmpDir; private Interval[] segmentIntervals; @@ -172,7 +172,7 @@ public int columnCacheSizeBytes() private void setupQueries() { // queries for the basic schema - BenchmarkSchemaInfo basicSchema = BenchmarkSchemas.SCHEMA_MAP.get("basic"); + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Collections.singletonList(basicSchema.getDataInterval())); @@ -291,7 +291,7 @@ public void setup() throws IOException setupQueries(); String schemaName = "basic"; - schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schemaName); + schemaInfo = GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName); segmentIntervals = new Interval[numSegments]; long startMillis = schemaInfo.getDataInterval().getStartMillis(); @@ -308,7 +308,7 @@ public void setup() throws IOException for (int i = 0; i < numSegments; i++) { log.info("Generating rows for segment " + i); - BenchmarkDataGenerator gen = new BenchmarkDataGenerator( + DataGenerator gen = new DataGenerator( schemaInfo.getColumnSchemas(), RNG_SEED + i, segmentIntervals[i], diff --git a/codestyle/druid-forbidden-apis.txt b/codestyle/druid-forbidden-apis.txt index 0ae33f5459ad..95c7d770326d 100644 --- a/codestyle/druid-forbidden-apis.txt +++ b/codestyle/druid-forbidden-apis.txt @@ -24,7 +24,6 @@ com.google.common.io.Files#createTempDir() @ Use org.apache.druid.java.util.comm com.google.common.util.concurrent.MoreExecutors#sameThreadExecutor() @ Use org.apache.druid.java.util.common.concurrent.Execs#directExecutor() com.google.common.util.concurrent.MoreExecutors#newDirectExecutorService() @ Use org.apache.druid.java.util.common.concurrent.Execs#directExecutor() com.google.common.util.concurrent.MoreExecutors#directExecutor() @ Use org.apache.druid.java.util.common.concurrent.Execs#directExecutor() -com.google.common.util.concurrent.Futures#transform(com.google.common.util.concurrent.ListenableFuture, com.google.common.util.concurrent.AsyncFunction) @ Use org.apache.druid.java.util.common.concurrent.ListenableFutures#transformAsync java.io.File#toURL() @ Use java.io.File#toURI() and java.net.URI#toURL() instead java.lang.String#matches(java.lang.String) @ Use startsWith(), endsWith(), contains(), or compile and cache a Pattern explicitly java.lang.String#replace(java.lang.CharSequence,java.lang.CharSequence) @ Use one of the appropriate methods in StringUtils instead diff --git a/core/pom.xml b/core/pom.xml index 627b3af4b80f..839d19b480c4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -394,6 +394,7 @@ @{jacocoArgLine} -Djava.library.path=${project.build.directory}/hyperic-sigar-${sigar.base.version}/sigar-bin/lib/ + -Duser.language=en diff --git a/core/src/main/java/org/apache/druid/collections/BlockingPool.java b/core/src/main/java/org/apache/druid/collections/BlockingPool.java index 91c3b35b6549..c17329917cd2 100644 --- a/core/src/main/java/org/apache/druid/collections/BlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/BlockingPool.java @@ -19,31 +19,12 @@ package org.apache.druid.collections; -import javax.annotation.Nullable; import java.util.List; public interface BlockingPool { int maxSize(); - /** - * Take a resource from the pool, waiting up to the - * specified wait time if necessary for an element to become available. - * - * @param timeoutMs maximum time to wait for a resource, in milliseconds. - * - * @return a resource, or null if the timeout was reached - */ - @Nullable - ReferenceCountingResourceHolder take(long timeoutMs); - - /** - * Take a resource from the pool, waiting if necessary until an element becomes available. - * - * @return a resource - */ - ReferenceCountingResourceHolder take(); - /** * Take resources from the pool, waiting up to the * specified wait time if necessary for elements of the given number to become available. diff --git a/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java b/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java index b1b43fd22477..a32236db20db 100644 --- a/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/DefaultBlockingPool.java @@ -76,32 +76,6 @@ public int getPoolSize() return objects.size(); } - @Override - @Nullable - public ReferenceCountingResourceHolder take(final long timeoutMs) - { - Preconditions.checkArgument(timeoutMs >= 0, "timeoutMs must be a non-negative value, but was [%s]", timeoutMs); - checkInitialized(); - try { - return wrapObject(timeoutMs > 0 ? pollObject(timeoutMs) : pollObject()); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Override - public ReferenceCountingResourceHolder take() - { - checkInitialized(); - try { - return wrapObject(takeObject()); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - @Nullable private ReferenceCountingResourceHolder wrapObject(T theObject) { @@ -144,21 +118,6 @@ private T pollObject(long timeoutMs) throws InterruptedException } } - private T takeObject() throws InterruptedException - { - final ReentrantLock lock = this.lock; - lock.lockInterruptibly(); - try { - while (objects.isEmpty()) { - notEnough.await(); - } - return objects.pop(); - } - finally { - lock.unlock(); - } - } - @Override public List> takeBatch(final int elementNum, final long timeoutMs) { diff --git a/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java b/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java index 037a489e1f19..dcd6cea07aa7 100644 --- a/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java +++ b/core/src/main/java/org/apache/druid/collections/DummyBlockingPool.java @@ -44,18 +44,6 @@ public int maxSize() return 0; } - @Override - public ReferenceCountingResourceHolder take(long timeoutMs) - { - throw new UnsupportedOperationException(); - } - - @Override - public ReferenceCountingResourceHolder take() - { - throw new UnsupportedOperationException(); - } - @Override public List> takeBatch(int elementNum, long timeoutMs) { diff --git a/core/src/main/java/org/apache/druid/collections/StupidPool.java b/core/src/main/java/org/apache/druid/collections/StupidPool.java index a69a8fde30a9..d1d6a9b9b7b1 100644 --- a/core/src/main/java/org/apache/druid/collections/StupidPool.java +++ b/core/src/main/java/org/apache/druid/collections/StupidPool.java @@ -34,6 +34,7 @@ import java.util.concurrent.atomic.AtomicReference; /** + * */ public class StupidPool implements NonBlockingPool { @@ -61,6 +62,7 @@ public class StupidPool implements NonBlockingPool private final String name; private final Supplier generator; + private final AtomicLong createdObjectsCounter = new AtomicLong(0); private final AtomicLong leakedObjectsCounter = new AtomicLong(0); //note that this is just the max entries in the cache, pool can still create as many buffers as needed. @@ -114,6 +116,7 @@ public ResourceHolder take() private ObjectResourceHolder makeObjectWithHandler() { T object = generator.get(); + createdObjectsCounter.incrementAndGet(); ObjectId objectId = new ObjectId(); ObjectLeakNotifier notifier = new ObjectLeakNotifier(this); // Using objectId as referent for Cleaner, because if the object itself (e. g. ByteBuffer) is leaked after taken @@ -122,7 +125,7 @@ private ObjectResourceHolder makeObjectWithHandler() } @VisibleForTesting - long poolSize() + public long poolSize() { return poolSize.get(); } @@ -133,6 +136,12 @@ long leakedObjectsCount() return leakedObjectsCounter.get(); } + @VisibleForTesting + public long objectsCreatedCount() + { + return createdObjectsCounter.get(); + } + private void tryReturnToPool(T object, ObjectId objectId, Cleaners.Cleanable cleanable, ObjectLeakNotifier notifier) { long currentPoolSize; @@ -160,7 +169,12 @@ private void tryReturnToPool(T object, ObjectId objectId, Cleaners.Cleanable cle * This should be impossible, because {@link ConcurrentLinkedQueue#offer(Object)} event don't have `return false;` in * it's body in OpenJDK 8. */ - private void impossibleOffsetFailed(T object, ObjectId objectId, Cleaners.Cleanable cleanable, ObjectLeakNotifier notifier) + private void impossibleOffsetFailed( + T object, + ObjectId objectId, + Cleaners.Cleanable cleanable, + ObjectLeakNotifier notifier + ) { poolSize.decrementAndGet(); notifier.disable(); diff --git a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java index 9e9a7d77df57..b779fc29d356 100644 --- a/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java +++ b/core/src/main/java/org/apache/druid/common/guava/CombiningSequence.java @@ -29,8 +29,6 @@ import java.util.Comparator; import java.util.function.BinaryOperator; -/** - */ public class CombiningSequence implements Sequence { public static CombiningSequence create( @@ -76,9 +74,22 @@ public Yielder toYielder(OutType initValue, final YieldingAcc new CombiningYieldingAccumulator<>(ordering, mergeFn, accumulator); combiningAccumulator.setRetVal(initValue); - Yielder baseYielder = baseSequence.toYielder(null, combiningAccumulator); - return makeYielder(baseYielder, combiningAccumulator, false); + final Yielder baseYielder = baseSequence.toYielder(null, combiningAccumulator); + + try { + return makeYielder(baseYielder, combiningAccumulator, false); + } + catch (Throwable t1) { + try { + baseYielder.close(); + } + catch (Throwable t2) { + t1.addSuppressed(t2); + } + + throw t1; + } } private Yielder makeYielder( diff --git a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java index 2a241f66de7c..c03e5f6d8e84 100644 --- a/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java +++ b/core/src/main/java/org/apache/druid/data/input/impl/prefetch/JsonIterator.java @@ -27,11 +27,11 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.CloseableIterator; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; -import java.util.Iterator; import java.util.NoSuchElementException; /** @@ -39,7 +39,7 @@ * * @param the type of object returned by this iterator */ -public class JsonIterator implements Iterator, Closeable +public class JsonIterator implements CloseableIterator { private JsonParser jp; private ObjectCodec objectCodec; diff --git a/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java b/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java index 33a4e3c74d58..bf9540015d33 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/StringUtils.java @@ -21,6 +21,7 @@ import com.google.common.base.Strings; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.UnsupportedEncodingException; import java.net.URLDecoder; @@ -472,28 +473,34 @@ public static String repeat(String s, int count) /** * Returns the string left-padded with the string pad to a length of len characters. * If str is longer than len, the return value is shortened to len characters. - * Lpad and rpad functions are migrated from flink's scala function with minor refactor + * This function is migrated from flink's scala function with minor refactor * https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala + * - Modified to handle empty pad string. + * - Padding of negative length return an empty string. * * @param base The base string to be padded * @param len The length of padded string * @param pad The pad string * - * @return the string left-padded with pad to a length of len + * @return the string left-padded with pad to a length of len or null if the pad is empty or the len is less than 0. */ - public static String lpad(String base, Integer len, String pad) + @Nonnull + public static String lpad(@Nonnull String base, int len, @Nonnull String pad) { - if (len < 0) { - return null; - } else if (len == 0) { + if (len <= 0) { return ""; } - char[] data = new char[len]; - // The length of the padding needed int pos = Math.max(len - base.length(), 0); + // short-circuit if there is no pad and we need to add a padding + if (pos > 0 && pad.isEmpty()) { + return base; + } + + char[] data = new char[len]; + // Copy the padding for (int i = 0; i < pos; i += pad.length()) { for (int j = 0; j < pad.length() && j < pos - i; j++) { @@ -512,37 +519,48 @@ public static String lpad(String base, Integer len, String pad) /** * Returns the string right-padded with the string pad to a length of len characters. * If str is longer than len, the return value is shortened to len characters. + * This function is migrated from flink's scala function with minor refactor + * https://github.com/apache/flink/blob/master/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/runtime/functions/ScalarFunctions.scala + * - Modified to handle empty pad string. + * - Modified to only copy the pad string if needed (this implementation mimics lpad). + * - Padding of negative length return an empty string. * * @param base The base string to be padded * @param len The length of padded string * @param pad The pad string * - * @return the string right-padded with pad to a length of len + * @return the string right-padded with pad to a length of len or null if the pad is empty or the len is less than 0. */ - public static String rpad(String base, Integer len, String pad) + @Nonnull + public static String rpad(@Nonnull String base, int len, @Nonnull String pad) { - if (len < 0) { - return null; - } else if (len == 0) { + if (len <= 0) { return ""; } - char[] data = new char[len]; - - int pos = 0; + // The length of the padding needed + int paddingLen = Math.max(len - base.length(), 0); - // Copy the base - for (; pos < base.length() && pos < len; pos++) { - data[pos] = base.charAt(pos); + // short-circuit if there is no pad and we need to add a padding + if (paddingLen > 0 && pad.isEmpty()) { + return base; } + char[] data = new char[len]; + + // Copy the padding - for (; pos < len; pos += pad.length()) { - for (int i = 0; i < pad.length() && i < len - pos; i++) { - data[pos + i] = pad.charAt(i); + for (int i = len - paddingLen; i < len; i += pad.length()) { + for (int j = 0; j < pad.length() && i + j < data.length; j++) { + data[i + j] = pad.charAt(j); } } + // Copy the base + for (int i = 0; i < len && i < base.length(); i++) { + data[i] = base.charAt(i); + } + return new String(data); } diff --git a/core/src/main/java/org/apache/druid/java/util/common/concurrent/ListenableFutures.java b/core/src/main/java/org/apache/druid/java/util/common/concurrent/ListenableFutures.java deleted file mode 100644 index 1722e4a9c663..000000000000 --- a/core/src/main/java/org/apache/druid/java/util/common/concurrent/ListenableFutures.java +++ /dev/null @@ -1,75 +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.java.util.common.concurrent; - -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.SettableFuture; - -import javax.annotation.Nullable; -import java.util.function.Function; - -public class ListenableFutures -{ - /** - * Guava 19 changes the Futures.transform signature so that the async form is different. This is here as a - * compatability layer until such a time as druid only supports Guava 19 or later, in which case - * Futures.transformAsync should be used - * - * This is NOT copied from guava. - */ - public static ListenableFuture transformAsync( - final ListenableFuture inFuture, - final Function> transform - ) - { - final SettableFuture finalFuture = SettableFuture.create(); - Futures.addCallback(inFuture, new FutureCallback() - { - @Override - public void onSuccess(@Nullable I result) - { - final ListenableFuture transformFuture = transform.apply(result); - Futures.addCallback(transformFuture, new FutureCallback() - { - @Override - public void onSuccess(@Nullable O result) - { - finalFuture.set(result); - } - - @Override - public void onFailure(Throwable t) - { - finalFuture.setException(t); - } - }); - } - - @Override - public void onFailure(Throwable t) - { - finalFuture.setException(t); - } - }); - return finalFuture; - } -} diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/ExplodingSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/ExplodingSequence.java new file mode 100644 index 000000000000..5468a5d86033 --- /dev/null +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/ExplodingSequence.java @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.java.util.common.guava; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicLong; + +/** + * Wraps an underlying sequence and allows us to force it to explode at various points. + */ +public class ExplodingSequence extends YieldingSequenceBase +{ + private final Sequence baseSequence; + private final boolean getThrowsException; + private final boolean closeThrowsException; + private final AtomicLong closed = new AtomicLong(); + + public ExplodingSequence(Sequence baseSequence, boolean getThrowsException, boolean closeThrowsException) + { + this.baseSequence = baseSequence; + this.getThrowsException = getThrowsException; + this.closeThrowsException = closeThrowsException; + } + + @Override + public Yielder toYielder( + final OutType initValue, + final YieldingAccumulator accumulator + ) + { + return wrapYielder(baseSequence.toYielder(initValue, accumulator)); + } + + public long getCloseCount() + { + return closed.get(); + } + + private Yielder wrapYielder(final Yielder baseYielder) + { + return new Yielder() + { + @Override + public OutType get() + { + if (getThrowsException) { + throw new RuntimeException("get"); + } else { + return baseYielder.get(); + } + } + + @Override + public Yielder next(OutType initValue) + { + return wrapYielder(baseYielder.next(initValue)); + } + + @Override + public boolean isDone() + { + return baseYielder.isDone(); + } + + @Override + public void close() throws IOException + { + closed.incrementAndGet(); + + if (closeThrowsException) { + throw new IOException("close"); + } else { + baseYielder.close(); + } + } + }; + } +} diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java index 912775cba8ed..90f9b9a48ff3 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/MergeSequence.java @@ -58,38 +58,62 @@ public Yielder toYielder(OutType initValue, YieldingAccumulat ) ); - pQueue = baseSequences.accumulate( - pQueue, - (queue, in) -> { - final Yielder yielder = in.toYielder( - null, - new YieldingAccumulator() - { - @Override - public T accumulate(T accumulated, T in) + try { + pQueue = baseSequences.accumulate( + pQueue, + (queue, in) -> { + final Yielder yielder = in.toYielder( + null, + new YieldingAccumulator() { - yield(); - return in; + @Override + public T accumulate(T accumulated, T in) + { + yield(); + return in; + } } + ); + + if (!yielder.isDone()) { + try { + queue.add(yielder); } - ); + catch (Throwable t1) { + try { + yielder.close(); + } + catch (Throwable t2) { + t1.addSuppressed(t2); + } - if (!yielder.isDone()) { - queue.add(yielder); - } else { - try { - yielder.close(); - } - catch (IOException e) { - throw new RuntimeException(e); + throw t1; + } + } else { + try { + yielder.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } } + + return queue; } + ); - return queue; - } - ); + return makeYielder(pQueue, initValue, accumulator); + } + catch (Throwable t1) { + try { + closeAll(pQueue); + } + catch (Throwable t2) { + t1.addSuppressed(t2); + } - return makeYielder(pQueue, initValue, accumulator); + throw t1; + } } private Yielder makeYielder( @@ -101,8 +125,22 @@ private Yielder makeYielder( OutType retVal = initVal; while (!accumulator.yielded() && !pQueue.isEmpty()) { Yielder yielder = pQueue.remove(); - retVal = accumulator.accumulate(retVal, yielder.get()); - yielder = yielder.next(null); + + try { + retVal = accumulator.accumulate(retVal, yielder.get()); + yielder = yielder.next(null); + } + catch (Throwable t1) { + try { + yielder.close(); + } + catch (Throwable t2) { + t1.addSuppressed(t2); + } + + throw t1; + } + if (yielder.isDone()) { try { yielder.close(); @@ -144,12 +182,21 @@ public boolean isDone() @Override public void close() throws IOException { - Closer closer = Closer.create(); - while (!pQueue.isEmpty()) { - closer.register(pQueue.remove()); - } - closer.close(); + closeAll(pQueue); } }; } + + private static void closeAll(final PriorityQueue> pQueue) throws IOException + { + Closer closer = Closer.create(); + while (!pQueue.isEmpty()) { + final Yielder yielder = pQueue.poll(); + if (yielder != null) { + // Note: yielder can be null if our comparator threw an exception during queue.add. + closer.register(yielder); + } + } + closer.close(); + } } diff --git a/core/src/main/java/org/apache/druid/java/util/common/guava/YieldingSequenceBase.java b/core/src/main/java/org/apache/druid/java/util/common/guava/YieldingSequenceBase.java index c5ed0111d074..9247989f5f64 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/guava/YieldingSequenceBase.java +++ b/core/src/main/java/org/apache/druid/java/util/common/guava/YieldingSequenceBase.java @@ -19,6 +19,8 @@ package org.apache.druid.java.util.common.guava; +import java.io.IOException; + /** * A Sequence that is based entirely on the Yielder implementation. *

@@ -29,16 +31,33 @@ public abstract class YieldingSequenceBase implements Sequence @Override public OutType accumulate(OutType initValue, Accumulator accumulator) { + final OutType retVal; Yielder yielder = toYielder(initValue, YieldingAccumulators.fromAccumulator(accumulator)); try { while (!yielder.isDone()) { yielder = yielder.next(yielder.get()); } - return yielder.get(); + retVal = yielder.get(); } - finally { - CloseQuietly.close(yielder); + catch (Throwable t1) { + try { + yielder.close(); + } + catch (Throwable t2) { + t1.addSuppressed(t2); + } + + throw t1; } + + try { + yielder.close(); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + return retVal; } } diff --git a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java index d5915369eada..af1baafe4198 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java +++ b/core/src/main/java/org/apache/druid/java/util/common/parsers/CloseableIterator.java @@ -66,10 +66,10 @@ default CloseableIterator flatMap(Function> funct return new CloseableIterator() { - CloseableIterator iterator = findNextIeteratorIfNecessary(); + CloseableIterator iterator = findNextIteratorIfNecessary(); @Nullable - private CloseableIterator findNextIeteratorIfNecessary() + private CloseableIterator findNextIteratorIfNecessary() { while ((iterator == null || !iterator.hasNext()) && delegate.hasNext()) { if (iterator != null) { @@ -105,7 +105,7 @@ public R next() return iterator.next(); } finally { - findNextIeteratorIfNecessary(); + findNextIteratorIfNecessary(); } } diff --git a/core/src/main/java/org/apache/druid/math/expr/Function.java b/core/src/main/java/org/apache/druid/math/expr/Function.java index af52555793b7..a20863929875 100644 --- a/core/src/main/java/org/apache/druid/math/expr/Function.java +++ b/core/src/main/java/org/apache/druid/math/expr/Function.java @@ -694,6 +694,11 @@ protected ExprEval eval(double param) class Round implements Function { + //CHECKSTYLE.OFF: Regexp + private static final BigDecimal MAX_FINITE_VALUE = BigDecimal.valueOf(Double.MAX_VALUE); + private static final BigDecimal MIN_FINITE_VALUE = BigDecimal.valueOf(-1 * Double.MAX_VALUE); + //CHECKSTYLE.ON: Regexp + @Override public String name() { @@ -705,7 +710,11 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) { ExprEval value1 = args.get(0).eval(bindings); if (value1.type() != ExprType.LONG && value1.type() != ExprType.DOUBLE) { - throw new IAE("The first argument to the function[%s] should be integer or double type but get the %s type", name(), value1.type()); + throw new IAE( + "The first argument to the function[%s] should be integer or double type but got the type: %s", + name(), + value1.type() + ); } if (args.size() == 1) { @@ -713,7 +722,11 @@ public ExprEval apply(List args, Expr.ObjectBinding bindings) } else { ExprEval value2 = args.get(1).eval(bindings); if (value2.type() != ExprType.LONG) { - throw new IAE("The second argument to the function[%s] should be integer type but get the %s type", name(), value2.type()); + throw new IAE( + "The second argument to the function[%s] should be integer type but got the type: %s", + name(), + value2.type() + ); } return eval(value1, value2.asInt()); } @@ -737,11 +750,27 @@ private ExprEval eval(ExprEval param, int scale) if (param.type() == ExprType.LONG) { return ExprEval.of(BigDecimal.valueOf(param.asLong()).setScale(scale, RoundingMode.HALF_UP).longValue()); } else if (param.type() == ExprType.DOUBLE) { - return ExprEval.of(BigDecimal.valueOf(param.asDouble()).setScale(scale, RoundingMode.HALF_UP).doubleValue()); + BigDecimal decimal = safeGetFromDouble(param.asDouble()); + return ExprEval.of(decimal.setScale(scale, RoundingMode.HALF_UP).doubleValue()); } else { return ExprEval.of(null); } } + + /** + * Converts non-finite doubles to BigDecimal values instead of throwing a NumberFormatException. + */ + private static BigDecimal safeGetFromDouble(double val) + { + if (Double.isNaN(val)) { + return BigDecimal.ZERO; + } else if (val == Double.POSITIVE_INFINITY) { + return MAX_FINITE_VALUE; + } else if (val == Double.NEGATIVE_INFINITY) { + return MIN_FINITE_VALUE; + } + return BigDecimal.valueOf(val); + } } class Signum extends UnivariateMathFunction diff --git a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java index 7e9eb9234b76..98c75505c05f 100644 --- a/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java +++ b/core/src/main/java/org/apache/druid/segment/loading/DataSegmentPusher.java @@ -20,9 +20,11 @@ package org.apache.druid.segment.loading; import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import java.io.File; import java.io.IOException; @@ -105,6 +107,13 @@ default List getAllowedPropertyPrefixesForHadoop() // on segment deletion if segment being deleted was the only segment static String getDefaultStorageDir(DataSegment segment, boolean useUniquePath) { + // Sanity check for shardSpec type. + // BucketNumberedShardSpec should never be used in segment push. + Preconditions.checkArgument( + !(segment.getShardSpec() instanceof BucketNumberedShardSpec), + "Illegal shardSpec type[%s]", + segment.getShardSpec() + ); return JOINER.join( segment.getDataSource(), StringUtils.format("%s_%s", segment.getInterval().getStart(), segment.getInterval().getEnd()), diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.java new file mode 100644 index 000000000000..d692dad113da --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BucketNumberedShardSpec.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.timeline.partition; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.RangeSet; + +import java.util.List; +import java.util.Map; + +/** + * This is one of the special shardSpecs which are temporarily used during batch ingestion. In Druid, there is a + * concept of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core + * partition set is represented as a range of partitionIds, i.e., [0, {@link ShardSpec#getNumCorePartitions()}). + * + * When you run a batch ingestion task with a non-linear partitioning scheme, the task populates all possible buckets + * upfront at the beginning (see {@code CachingLocalSegmentAllocator}) and uses them to partition input rows. However, + * some of the buckets can be empty even after the task consumes all inputs if the data is highly skewed. Since Druid + * doesn't create empty segments, the partitionId should be dynamically allocated when a bucket is actually in use, + * so that we can always create the packed core partition set without missing partitionIds. + * + * This BucketNumberedShardSpec is used for such use case. The task with a non-linear partitioning scheme uses it + * to postpone the partitionId allocation until all empty buckets are identified. See + * {@code ParallelIndexSupervisorTask.groupGenericPartitionLocationsPerPartition} and + * {@code CachingLocalSegmentAllocator} for parallel and sequential ingestion, respectively. + * + * Note that {@link org.apache.druid.timeline.SegmentId} requires the partitionId. Since the segmentId is used + * everwhere during ingestion, this class should implement {@link #getPartitionNum()} which returns the bucketId. + * This should be fine because the segmentId is only used to identify each segment until pushing them to deep storage. + * The bucketId should be enough to uniquely identify each segment. However, when pushing segments to deep storage, + * the partitionId is used to create the path to store the segment on deep storage + * ({@link org.apache.druid.segment.loading.DataSegmentPusher#getDefaultStorageDir} which should be correct. + * As a result, this shardSpec should not be used in pushing segments. + * + * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. + * + * This interface doesn't really have to extend {@link ShardSpec}. The only reason is the ShardSpec is used in many + * places such as {@link org.apache.druid.timeline.DataSegment}, and we have to modify those places to allow other + * types than ShardSpec which seems pretty invasive. Maybe we could clean up this mess someday in the future. + * + * @see BuildingShardSpec + */ +public interface BucketNumberedShardSpec extends ShardSpec +{ + int getBucketId(); + + T convert(int partitionId); + + @Override + default PartitionChunk createChunk(O obj) + { + // The partitionId (or partitionNum, chunkNumber) is not determined yet. Use bucketId for now. + return new NumberedPartitionChunk<>(getBucketId(), 0, obj); + } + + @Override + default int getPartitionNum() + { + // See the class-level Javadoc for returning bucketId here. + return getBucketId(); + } + + @Override + default int getNumCorePartitions() + { + throw new UnsupportedOperationException(); + } + + // The below methods are used on the query side, and so must not be called for this shardSpec. + + @JsonIgnore + @Override + default List getDomainDimensions() + { + throw new UnsupportedOperationException(); + } + + @Override + default boolean possibleInDomain(Map> domain) + { + throw new UnsupportedOperationException(); + } + + @Override + default boolean isCompatible(Class other) + { + throw new UnsupportedOperationException(); + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java new file mode 100644 index 000000000000..fb896fc2ac84 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpec.java @@ -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. + */ + +package org.apache.druid.timeline.partition; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +/** + * See {@link BuildingShardSpec} for how this class is used. + * + * @see HashBasedNumberedShardSpec + */ +public class BuildingHashBasedNumberedShardSpec implements BuildingShardSpec +{ + public static final String TYPE = "building_hashed"; + + private final int partitionId; + private final int bucketId; + private final int numBuckets; + private final List partitionDimensions; + private final ObjectMapper jsonMapper; + + @JsonCreator + public BuildingHashBasedNumberedShardSpec( + @JsonProperty("partitionId") int partitionId, + @JsonProperty("bucketId") int bucketId, + @JsonProperty("numBuckets") int numBuckets, + @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, + @JacksonInject ObjectMapper jsonMapper + ) + { + this.partitionId = partitionId; + this.bucketId = bucketId; + this.numBuckets = numBuckets; + this.partitionDimensions = partitionDimensions == null + ? HashBasedNumberedShardSpec.DEFAULT_PARTITION_DIMENSIONS + : partitionDimensions; + this.jsonMapper = jsonMapper; + } + + @JsonProperty("partitionId") + @Override + public int getPartitionNum() + { + return partitionId; + } + + @Override + @JsonProperty + public int getBucketId() + { + return bucketId; + } + + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + + @JsonProperty + public List getPartitionDimensions() + { + return partitionDimensions; + } + + @Override + public PartitionChunk createChunk(T obj) + { + // This method can be called in AppenderatorImpl to create a sinkTimeline. + // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + // HashBasedNumberedShardSpec is using NumberedPartitionChunk, so we use it here too. + return new NumberedPartitionChunk<>(partitionId, 0, obj); + } + + @Override + public HashBasedNumberedShardSpec convert(int numCorePartitions) + { + return new HashBasedNumberedShardSpec( + partitionId, + numCorePartitions, + bucketId, + numBuckets, + partitionDimensions, + jsonMapper + ); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BuildingHashBasedNumberedShardSpec that = (BuildingHashBasedNumberedShardSpec) o; + return partitionId == that.partitionId && + bucketId == that.bucketId && + numBuckets == that.numBuckets && + Objects.equals(partitionDimensions, that.partitionDimensions); + } + + @Override + public int hashCode() + { + return Objects.hash(partitionId, bucketId, numBuckets, partitionDimensions); + } + + @Override + public String toString() + { + return "BuildingHashBasedNumberedShardSpec{" + + "partitionId=" + partitionId + + ", bucketId=" + bucketId + + ", numBuckets=" + numBuckets + + ", partitionDimensions=" + partitionDimensions + + '}'; + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.java new file mode 100644 index 000000000000..a179d3ca7bac --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpec.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.timeline.partition; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import java.util.Objects; + +/** + * See {@link BuildingShardSpec} for how this class is used. + * + * This shardSpec has only partitionId which is same as {@link LinearShardSpec}. The difference between + * them is this shardSpec should never be published and so never be used in other places such as Broker timeline. + * + * @see NumberedShardSpec + */ +public class BuildingNumberedShardSpec implements BuildingShardSpec +{ + public static final String TYPE = "building_numbered"; + + private final int partitionId; + + @JsonCreator + public BuildingNumberedShardSpec(@JsonProperty("partitionId") int partitionId) + { + Preconditions.checkArgument(partitionId >= 0, "partitionId >= 0"); + this.partitionId = partitionId; + } + + @Override + public int getBucketId() + { + // This method is currently not called when the shardSpec type is this class. + throw new UnsupportedOperationException(); + } + + @Override + public NumberedShardSpec convert(int numTotalPartitions) + { + return new NumberedShardSpec(partitionId, numTotalPartitions); + } + + @Override + public PartitionChunk createChunk(T obj) + { + // This method can be called in AppenderatorImpl to create a sinkTimeline. + // The sinkTimeline doesn't seem in use in batch ingestion, let's set 'chunks' to 0 for now. + return new NumberedPartitionChunk<>(partitionId, 0, obj); + } + + @JsonProperty("partitionId") + @Override + public int getPartitionNum() + { + return partitionId; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BuildingNumberedShardSpec shardSpec = (BuildingNumberedShardSpec) o; + return partitionId == shardSpec.partitionId; + } + + @Override + public int hashCode() + { + return Objects.hash(partitionId); + } + + @Override + public String toString() + { + return "BuildingNumberedShardSpec{" + + "partitionId=" + partitionId + + '}'; + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.java new file mode 100644 index 000000000000..973fdf4d2a7d --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingShardSpec.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.timeline.partition; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.RangeSet; +import org.apache.druid.data.input.InputRow; + +import java.util.List; +import java.util.Map; + +/** + * This is one of the special shardSpecs which are temporarily used during batch ingestion. In Druid, there is a + * concept of core partition set which is a set of segments atomically becoming queryable together in Brokers. The core + * partition set is represented as a range of partitionIds, i.e., [0, {@link ShardSpec#getNumCorePartitions()}). + * + * In streaming ingestion, the core partition set size cannot be determined since it's impossible to know how many + * segments will be created per time chunk upfront. However, in batch ingestion with time chunk locking, the core + * partition set is the set of segments created by an initial task or an overwriting task. Since the core partition + * set is determined when the task publishes segments at the end, the task postpones creating proper {@link ShardSpec} + * until the end. + * + * This BuildingShardSpec is used for such use case. A non-appending batch task can use this shardSpec until it + * publishes segments at last. When it publishes segments, it should convert the buildingShardSpec of those segments + * to a proper shardSpec type {@link T}. See {@code SegmentPublisherHelper#annotateShardSpec} for converting shardSpec. + * Note that, when the segment lock is used, the Overlord coordinates the segment allocation and this class is never + * used. Instead, the task sends {@link PartialShardSpec} to the Overlord to allocate a new segment. The result segment + * could have either a {@link ShardSpec} (for root generation segments) or an {@link OverwriteShardSpec} (for non-root + * generation segments). + * + * This class should be Jackson-serializable as the subtasks can send it to the parallel task in parallel ingestion. + * + * This interface doesn't really have to extend {@link ShardSpec}. The only reason is the ShardSpec is used in many + * places such as {@link org.apache.druid.timeline.DataSegment}, and we have to modify those places to allow other + * types than ShardSpec which seems pretty invasive. Maybe we could clean up this mess someday in the future. + * + * @see BucketNumberedShardSpec + */ +public interface BuildingShardSpec extends ShardSpec +{ + int getBucketId(); + + T convert(int numCorePartitions); + + @Override + default int getNumCorePartitions() + { + throw new UnsupportedOperationException(); + } + + /** + * {@link BucketNumberedShardSpec} should be used for shard spec lookup. + */ + @Override + default ShardSpecLookup getLookup(List shardSpecs) + { + throw new UnsupportedOperationException(); + } + + // The below methods are used on the query side, and so must not be called for this shardSpec. + + @Override + default boolean isInChunk(long timestamp, InputRow inputRow) + { + throw new UnsupportedOperationException(); + } + + @JsonIgnore + @Override + default List getDomainDimensions() + { + throw new UnsupportedOperationException(); + } + + @Override + default boolean possibleInDomain(Map> domain) + { + throw new UnsupportedOperationException(); + } + + @Override + default boolean isCompatible(Class other) + { + throw new UnsupportedOperationException(); + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java new file mode 100644 index 000000000000..6dd099205447 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpec.java @@ -0,0 +1,141 @@ +/* + * 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.timeline.partition; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * See {@link BuildingShardSpec} for how this class is used. + * + * @see SingleDimensionShardSpec + */ +public class BuildingSingleDimensionShardSpec implements BuildingShardSpec +{ + public static final String TYPE = "building_single_dim"; + + private final int bucketId; + private final String dimension; + @Nullable + private final String start; + @Nullable + private final String end; + private final int partitionId; + + @JsonCreator + public BuildingSingleDimensionShardSpec( + @JsonProperty("bucketId") int bucketId, + @JsonProperty("dimension") String dimension, + @JsonProperty("start") @Nullable String start, + @JsonProperty("end") @Nullable String end, + @JsonProperty("partitionNum") int partitionNum + ) + { + this.bucketId = bucketId; + this.dimension = dimension; + this.start = start; + this.end = end; + this.partitionId = partitionNum; + } + + @JsonProperty("dimension") + public String getDimension() + { + return dimension; + } + + @Nullable + @JsonProperty("start") + public String getStart() + { + return start; + } + + @Nullable + @JsonProperty("end") + public String getEnd() + { + return end; + } + + @Override + @JsonProperty("partitionNum") + public int getPartitionNum() + { + return partitionId; + } + + @Override + @JsonProperty("bucketId") + public int getBucketId() + { + return bucketId; + } + + @Override + public SingleDimensionShardSpec convert(int numCorePartitions) + { + return new SingleDimensionShardSpec(dimension, start, end, partitionId, numCorePartitions); + } + + @Override + public PartitionChunk createChunk(T obj) + { + return new NumberedPartitionChunk<>(partitionId, 0, obj); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + BuildingSingleDimensionShardSpec that = (BuildingSingleDimensionShardSpec) o; + return bucketId == that.bucketId && + partitionId == that.partitionId && + Objects.equals(dimension, that.dimension) && + Objects.equals(start, that.start) && + Objects.equals(end, that.end); + } + + @Override + public int hashCode() + { + return Objects.hash(bucketId, dimension, start, end, partitionId); + } + + @Override + public String toString() + { + return "BuildingSingleDimensionShardSpec{" + + "bucketId=" + bucketId + + ", dimension='" + dimension + '\'' + + ", start='" + start + '\'' + + ", end='" + end + '\'' + + ", partitionNum=" + partitionId + + '}'; + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java index 495a85284bb3..0e32ee04bcf1 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpec.java @@ -33,15 +33,18 @@ public class HashBasedNumberedPartialShardSpec implements PartialShardSpec @Nullable private final List partitionDimensions; + private final int bucketId; private final int numBuckets; @JsonCreator public HashBasedNumberedPartialShardSpec( @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, + @JsonProperty("bucketId") int bucketId, @JsonProperty("numPartitions") int numBuckets ) { this.partitionDimensions = partitionDimensions; + this.bucketId = bucketId; this.numBuckets = numBuckets; } @@ -52,6 +55,12 @@ public List getPartitionDimensions() return partitionDimensions; } + @JsonProperty + public int getBucketId() + { + return bucketId; + } + @JsonProperty("numPartitions") public int getNumBuckets() { @@ -61,9 +70,16 @@ public int getNumBuckets() @Override public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { - final HashBasedNumberedShardSpec prevSpec = (HashBasedNumberedShardSpec) specOfPreviousMaxPartitionId; + // The shardSpec is created by the Overlord. + // For batch tasks, this code can be executed only with segment locking (forceTimeChunkLock = false). + // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of + // the same datasource. Since there is no restriction for those tasks in segment allocation, the + // allocated IDs for each task can interleave. As a result, the core partition set cannot be + // represented as a range. We always set 0 for the core partition set size if this is an initial segment. return new HashBasedNumberedShardSpec( - prevSpec == null ? 0 : prevSpec.getPartitionNum() + 1, + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1, + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getNumCorePartitions(), + bucketId, numBuckets, partitionDimensions, objectMapper @@ -73,7 +89,7 @@ public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfP @Override public ShardSpec complete(ObjectMapper objectMapper, int partitionId) { - return new HashBasedNumberedShardSpec(partitionId, numBuckets, partitionDimensions, objectMapper); + return new HashBasedNumberedShardSpec(partitionId, 0, bucketId, numBuckets, partitionDimensions, objectMapper); } @Override @@ -92,13 +108,14 @@ public boolean equals(Object o) return false; } HashBasedNumberedPartialShardSpec that = (HashBasedNumberedPartialShardSpec) o; - return numBuckets == that.numBuckets && + return bucketId == that.bucketId && + numBuckets == that.numBuckets && Objects.equals(partitionDimensions, that.partitionDimensions); } @Override public int hashCode() { - return Objects.hash(partitionDimensions, numBuckets); + return Objects.hash(partitionDimensions, bucketId, numBuckets); } } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java index a03fddceaf3a..23cdb4ef3868 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpec.java @@ -35,29 +35,55 @@ import javax.annotation.Nullable; import java.util.List; +import java.util.Objects; public class HashBasedNumberedShardSpec extends NumberedShardSpec { + static final List DEFAULT_PARTITION_DIMENSIONS = ImmutableList.of(); + private static final HashFunction HASH_FUNCTION = Hashing.murmur3_32(); - private static final List DEFAULT_PARTITION_DIMENSIONS = ImmutableList.of(); + private final int bucketId; + /** + * Number of hash buckets + */ + private final int numBuckets; private final ObjectMapper jsonMapper; @JsonIgnore private final List partitionDimensions; @JsonCreator public HashBasedNumberedShardSpec( - @JsonProperty("partitionNum") int partitionNum, // partitionId - @JsonProperty("partitions") int partitions, // # of partitions + @JsonProperty("partitionNum") int partitionNum, // partitionId, hash bucketId + @JsonProperty("partitions") int partitions, // core partition set size + @JsonProperty("bucketId") @Nullable Integer bucketId, // nullable for backward compatibility + @JsonProperty("numBuckets") @Nullable Integer numBuckets, // nullable for backward compatibility @JsonProperty("partitionDimensions") @Nullable List partitionDimensions, @JacksonInject ObjectMapper jsonMapper ) { super(partitionNum, partitions); + // Use partitionId as bucketId if it's missing. + this.bucketId = bucketId == null ? partitionNum : bucketId; + // If numBuckets is missing, assume that any hash bucket is not empty. + // Use the core partition set size as the number of buckets. + this.numBuckets = numBuckets == null ? partitions : numBuckets; this.jsonMapper = jsonMapper; this.partitionDimensions = partitionDimensions == null ? DEFAULT_PARTITION_DIMENSIONS : partitionDimensions; } + @JsonProperty + public int getBucketId() + { + return bucketId; + } + + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + @JsonProperty("partitionDimensions") public List getPartitionDimensions() { @@ -73,12 +99,27 @@ public boolean isCompatible(Class other) @Override public boolean isInChunk(long timestamp, InputRow inputRow) { - return (((long) hash(timestamp, inputRow)) - getPartitionNum()) % getPartitions() == 0; + return (((long) hash(timestamp, inputRow)) - bucketId) % numBuckets == 0; } + /** + * This method calculates the hash based on whether {@param partitionDimensions} is null or not. + * If yes, then both {@param timestamp} and dimension columns in {@param inputRow} are used {@link Rows#toGroupKey} + * Or else, columns in {@param partitionDimensions} are used + * + * @param timestamp should be bucketed with query granularity + * @param inputRow row from input data + * + * @return hash value + */ protected int hash(long timestamp, InputRow inputRow) { - final List groupKey = getGroupKey(timestamp, inputRow); + return hash(jsonMapper, partitionDimensions, timestamp, inputRow); + } + + public static int hash(ObjectMapper jsonMapper, List partitionDimensions, long timestamp, InputRow inputRow) + { + final List groupKey = getGroupKey(partitionDimensions, timestamp, inputRow); try { return hash(jsonMapper, groupKey); } @@ -88,7 +129,7 @@ protected int hash(long timestamp, InputRow inputRow) } @VisibleForTesting - List getGroupKey(final long timestamp, final InputRow inputRow) + static List getGroupKey(final List partitionDimensions, final long timestamp, final InputRow inputRow) { if (partitionDimensions.isEmpty()) { return Rows.toGroupKey(timestamp, inputRow); @@ -104,21 +145,57 @@ public static int hash(ObjectMapper jsonMapper, List objects) throws Jso } @Override - public String toString() + public ShardSpecLookup getLookup(final List shardSpecs) { - return "HashBasedNumberedShardSpec{" + - "partitionNum=" + getPartitionNum() + - ", partitions=" + getPartitions() + - ", partitionDimensions=" + getPartitionDimensions() + - '}'; + return createHashLookup(jsonMapper, partitionDimensions, shardSpecs, numBuckets); } - @Override - public ShardSpecLookup getLookup(final List shardSpecs) + static ShardSpecLookup createHashLookup( + ObjectMapper jsonMapper, + List partitionDimensions, + List shardSpecs, + int numBuckets + ) { return (long timestamp, InputRow row) -> { - int index = Math.abs(hash(timestamp, row) % getPartitions()); + int index = Math.abs(hash(jsonMapper, partitionDimensions, timestamp, row) % numBuckets); return shardSpecs.get(index); }; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + HashBasedNumberedShardSpec that = (HashBasedNumberedShardSpec) o; + return bucketId == that.bucketId && + numBuckets == that.numBuckets && + Objects.equals(partitionDimensions, that.partitionDimensions); + } + + @Override + public int hashCode() + { + return Objects.hash(super.hashCode(), bucketId, numBuckets, partitionDimensions); + } + + @Override + public String toString() + { + return "HashBasedNumberedShardSpec{" + + "partitionNum=" + getPartitionNum() + + ", partitions=" + getNumCorePartitions() + + ", bucketId=" + bucketId + + ", numBuckets=" + numBuckets + + ", partitionDimensions=" + partitionDimensions + + '}'; + } } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.java new file mode 100644 index 000000000000..324c02044985 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/HashBucketShardSpec.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.timeline.partition; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputRow; + +import java.util.List; +import java.util.Objects; + +/** + * See {@link BucketNumberedShardSpec} for how this class is used. + * + * @see BuildingHashBasedNumberedShardSpec + */ +public class HashBucketShardSpec implements BucketNumberedShardSpec +{ + public static final String TYPE = "bucket_hash"; + + private final int bucketId; + private final int numBuckets; + private final List partitionDimensions; + private final ObjectMapper jsonMapper; + + @JsonCreator + public HashBucketShardSpec( + @JsonProperty("bucketId") int bucketId, + @JsonProperty("numBuckets") int numBuckets, + @JsonProperty("partitionDimensions") List partitionDimensions, + @JacksonInject ObjectMapper jsonMapper + ) + { + this.bucketId = bucketId; + this.numBuckets = numBuckets; + this.partitionDimensions = partitionDimensions == null + ? HashBasedNumberedShardSpec.DEFAULT_PARTITION_DIMENSIONS + : partitionDimensions; + this.jsonMapper = jsonMapper; + } + + @Override + @JsonProperty + public int getBucketId() + { + return bucketId; + } + + @JsonProperty + public int getNumBuckets() + { + return numBuckets; + } + + @JsonProperty + public List getPartitionDimensions() + { + return partitionDimensions; + } + + @Override + public BuildingHashBasedNumberedShardSpec convert(int partitionId) + { + return new BuildingHashBasedNumberedShardSpec(partitionId, bucketId, numBuckets, partitionDimensions, jsonMapper); + } + + @Override + public boolean isInChunk(long timestamp, InputRow inputRow) + { + // not in use + throw new UnsupportedOperationException(); + } + + @Override + public ShardSpecLookup getLookup(List shardSpecs) + { + return HashBasedNumberedShardSpec.createHashLookup(jsonMapper, partitionDimensions, shardSpecs, numBuckets); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + HashBucketShardSpec that = (HashBucketShardSpec) o; + return bucketId == that.bucketId && + numBuckets == that.numBuckets && + Objects.equals(partitionDimensions, that.partitionDimensions); + } + + @Override + public int hashCode() + { + return Objects.hash(bucketId, numBuckets, partitionDimensions); + } + + @Override + public String toString() + { + return "HashBucket{" + + ", bucketId=" + bucketId + + ", numBuckets=" + numBuckets + + ", partitionDimensions=" + partitionDimensions + + '}'; + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java index 1ebb24e16038..95b0bd832b9b 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/LinearShardSpec.java @@ -50,7 +50,13 @@ public int getPartitionNum() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public int getNumCorePartitions() + { + return 0; + } + + @Override + public ShardSpecLookup getLookup(final List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java index dde92167362a..d53011819413 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NoneShardSpec.java @@ -66,7 +66,13 @@ public int getPartitionNum() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public int getNumCorePartitions() + { + return 0; + } + + @Override + public ShardSpecLookup getLookup(final List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java index d5b36576a654..adb0d288d6a6 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpec.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.RangeSet; import org.apache.druid.data.input.InputRow; +import org.apache.druid.timeline.DataSegment; import java.util.Collections; import java.util.List; @@ -31,10 +32,28 @@ import java.util.Objects; /** - * ShardSpec for segments which overshadow others with their minorVersion. + * This shardSpec is used only for the segments created by overwriting tasks with segment lock enabled. + * When the segment lock is used, there is a concept of atomic update group which is a set of segments atomically + * becoming queryable together in Brokers. It is a similar concept to the core partition set (explained + * {@link NumberedShardSpec}), but different in a sense that there is only one core partition set per time chunk + * while there could be multiple atomic update groups in one time chunk. + * + * The atomic update group has the root partition range and the minor version to determine the visibility between + * atomic update groups; the group of the highest minor version in the same root partition range becomes queryable + * when they have the same major version ({@link DataSegment#getVersion()}). + * + * Note that this shardSpec is used only when you overwrite existing segments with segment lock enabled. + * If the task doesn't overwrite segments, it will use NumberedShardSpec instead even when segment lock is used. + * Similar to NumberedShardSpec, the size of the atomic update group is determined when the task publishes segments + * at the end of ingestion. As a result, {@link #atomicUpdateGroupSize} is set to + * {@link PartitionIds#UNKNOWN_ATOMIC_UPDATE_GROUP_SIZE} first, and updated when publishing segments + * in {@code SegmentPublisherHelper#annotateShardSpec}. + * + * @see AtomicUpdateGroup */ public class NumberedOverwriteShardSpec implements OverwriteShardSpec { + public static final String TYPE = "numbered_overwrite"; private final int partitionId; private final short startRootPartitionId; @@ -169,7 +188,7 @@ public short getAtomicUpdateGroupSize() } @Override - public ShardSpecLookup getLookup(List shardSpecs) + public ShardSpecLookup getLookup(List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java index 0e258ee786b2..730502845f30 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedPartialShardSpec.java @@ -40,10 +40,17 @@ private NumberedPartialShardSpec() public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { if (specOfPreviousMaxPartitionId == null) { + // The shardSpec is created by the Overlord. + // - For streaming ingestion tasks, the core partition set is always 0. + // - For batch tasks, this code is executed only with segment locking (forceTimeChunkLock = false). + // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of + // the same datasource. Since there is no restriction for those tasks in segment allocation, the + // allocated IDs for each task can interleave. As a result, the core partition set cannot be + // represented as a range. We always set 0 for the core partition set size. return new NumberedShardSpec(0, 0); } else { final NumberedShardSpec prevSpec = (NumberedShardSpec) specOfPreviousMaxPartitionId; - return new NumberedShardSpec(prevSpec.getPartitionNum() + 1, prevSpec.getPartitions()); + return new NumberedShardSpec(prevSpec.getPartitionNum() + 1, prevSpec.getNumCorePartitions()); } } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java index d6f98d71751d..5db6bf6000ac 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/NumberedShardSpec.java @@ -66,7 +66,17 @@ public int getPartitionNum() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + public ShardSpecLookup getLookup(final List shardSpecs) + { + return createNumberedLookup(shardSpecs); + } + + static ShardSpecLookup createNumberedLookup(List shardSpecs) + { + return createLookup(shardSpecs); + } + + static ShardSpecLookup createLookup(List shardSpecs) { return (long timestamp, InputRow row) -> shardSpecs.get(0); } @@ -89,8 +99,9 @@ public boolean isCompatible(Class other) return other == NumberedShardSpec.class || other == NumberedOverwriteShardSpec.class; } + @Override @JsonProperty("partitions") - public int getPartitions() + public int getNumCorePartitions() { return partitions; } @@ -122,16 +133,12 @@ public boolean equals(Object o) if (this == o) { return true; } - - if (!(o instanceof NumberedShardSpec)) { - return false; - } - - final NumberedShardSpec that = (NumberedShardSpec) o; - if (partitionNum != that.partitionNum) { + if (o == null || getClass() != o.getClass()) { return false; } - return partitions == that.partitions; + NumberedShardSpec that = (NumberedShardSpec) o; + return partitionNum == that.partitionNum && + partitions == that.partitions; } @Override diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java index 0fea5646e311..6a77ea55286b 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/OverwriteShardSpec.java @@ -28,6 +28,17 @@ */ public interface OverwriteShardSpec extends ShardSpec { + /** + * The core partition concept is not used with segment locking. Instead, the {@link AtomicUpdateGroup} is used + * to atomically overshadow segments. Here, we always returns 0 so that the {@link PartitionHolder} skips checking + * the completeness of the core partitions. + */ + @Override + default int getNumCorePartitions() + { + return 0; + } + default OverwriteShardSpec withAtomicUpdateGroupSize(int atomicUpdateGroupSize) { return withAtomicUpdateGroupSize((short) atomicUpdateGroupSize); diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java index 9cbee2c7571c..6afaa939471e 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartialShardSpec.java @@ -28,8 +28,12 @@ import javax.annotation.Nullable; /** - * Class to contain all information of a {@link ShardSpec} except for the partition ID. - * This class is mainly used by the indexing tasks to allocate new segments using the Overlord. + * This interface is used in the segment allocation protocol when it is coordinated by the Overlord; when appending + * segments to an existing datasource (either streaming ingestion or batch append) or any case when segment + * lock is used. The implementations of this interface contain all information of the corresponding {@link ShardSpec} + * except the partition ID. + * The ingestion tasks send all information required for allocating a new segment using this interface and the Overlord + * determines the partition ID to create a new segment. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes({ diff --git a/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java new file mode 100644 index 000000000000..a329131e7487 --- /dev/null +++ b/core/src/main/java/org/apache/druid/timeline/partition/RangeBucketShardSpec.java @@ -0,0 +1,137 @@ +/* + * 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.timeline.partition; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.data.input.InputRow; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Objects; + +/** + * See {@link BucketNumberedShardSpec} for how this class is used. + * + * @see BuildingSingleDimensionShardSpec + */ +public class RangeBucketShardSpec implements BucketNumberedShardSpec +{ + public static final String TYPE = "bucket_single_dim"; + + private final int bucketId; + private final String dimension; + @Nullable + private final String start; + @Nullable + private final String end; + + @JsonCreator + public RangeBucketShardSpec( + @JsonProperty("bucketId") int bucketId, + @JsonProperty("dimension") String dimension, + @JsonProperty("start") @Nullable String start, + @JsonProperty("end") @Nullable String end + ) + { + this.bucketId = bucketId; + this.dimension = dimension; + this.start = start; + this.end = end; + } + + @Override + @JsonProperty + public int getBucketId() + { + return bucketId; + } + + @JsonProperty + public String getDimension() + { + return dimension; + } + + @Nullable + @JsonProperty + public String getStart() + { + return start; + } + + @Nullable + @JsonProperty + public String getEnd() + { + return end; + } + + @Override + public BuildingSingleDimensionShardSpec convert(int partitionId) + { + return new BuildingSingleDimensionShardSpec(bucketId, dimension, start, end, partitionId); + } + + @Override + public boolean isInChunk(long timestamp, InputRow inputRow) + { + return SingleDimensionShardSpec.isInChunk(dimension, start, end, inputRow); + } + + @Override + public ShardSpecLookup getLookup(List shardSpecs) + { + return SingleDimensionShardSpec.createLookup(shardSpecs); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RangeBucketShardSpec bucket = (RangeBucketShardSpec) o; + return bucketId == bucket.bucketId && + Objects.equals(dimension, bucket.dimension) && + Objects.equals(start, bucket.start) && + Objects.equals(end, bucket.end); + } + + @Override + public int hashCode() + { + return Objects.hash(bucketId, dimension, start, end); + } + + @Override + public String toString() + { + return "RangeBucket{" + + ", bucketId=" + bucketId + + ", dimension='" + dimension + '\'' + + ", start='" + start + '\'' + + ", end='" + end + '\'' + + '}'; + } +} diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java index 43aaf701db36..8f59d3959e72 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpec.java @@ -19,6 +19,7 @@ package org.apache.druid.timeline.partition; +import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.google.common.collect.RangeSet; @@ -38,53 +39,94 @@ @JsonSubTypes.Type(name = "linear", value = LinearShardSpec.class), @JsonSubTypes.Type(name = "numbered", value = NumberedShardSpec.class), @JsonSubTypes.Type(name = "hashed", value = HashBasedNumberedShardSpec.class), - @JsonSubTypes.Type(name = "numbered_overwrite", value = NumberedOverwriteShardSpec.class) + @JsonSubTypes.Type(name = NumberedOverwriteShardSpec.TYPE, value = NumberedOverwriteShardSpec.class), + // BuildingShardSpecs are the shardSpec with missing numCorePartitions, and thus must not be published. + // See BuildingShardSpec for more details. + @JsonSubTypes.Type(name = BuildingNumberedShardSpec.TYPE, value = BuildingNumberedShardSpec.class), + @JsonSubTypes.Type(name = BuildingHashBasedNumberedShardSpec.TYPE, value = BuildingHashBasedNumberedShardSpec.class), + @JsonSubTypes.Type(name = BuildingSingleDimensionShardSpec.TYPE, value = BuildingSingleDimensionShardSpec.class), + // BucketShardSpecs are the shardSpec with missing partitionId and numCorePartitions. + // These shardSpecs must not be used in segment push. + // See BucketShardSpec for more details. + @JsonSubTypes.Type(name = HashBucketShardSpec.TYPE, value = HashBucketShardSpec.class), + @JsonSubTypes.Type(name = RangeBucketShardSpec.TYPE, value = RangeBucketShardSpec.class) }) public interface ShardSpec { + @JsonIgnore PartitionChunk createChunk(T obj); + @JsonIgnore boolean isInChunk(long timestamp, InputRow inputRow); + /** + * Returns the partition ID of this segment. + */ int getPartitionNum(); + int getNumCorePartitions(); + + /** + * Returns the start root partition ID of the atomic update group which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default int getStartRootPartitionId() { return getPartitionNum(); } + /** + * Returns the end root partition ID of the atomic update group which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default int getEndRootPartitionId() { return getPartitionNum() + 1; } + /** + * Returns the minor version associated to the atomic update group which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default short getMinorVersion() { return 0; } + /** + * Returns the atomic update group size which this segment belongs to. + * + * @see AtomicUpdateGroup + */ default short getAtomicUpdateGroupSize() { return 1; } - ShardSpecLookup getLookup(List shardSpecs); + @JsonIgnore + ShardSpecLookup getLookup(List shardSpecs); /** * Get dimensions who have possible range for the rows this shard contains. * * @return list of dimensions who has its possible range. Dimensions with unknown possible range are not listed */ + @JsonIgnore List getDomainDimensions(); /** * if given domain ranges are not possible in this shard, return false; otherwise return true; * @return possibility of in domain */ + @JsonIgnore boolean possibleInDomain(Map> domain); /** - * Returns true if two segments of this and other shardSpecs can exist in the same timeChunk. + * Returns true if two segments of this and other shardSpecs can exist in the same time chunk. */ + @JsonIgnore boolean isCompatible(Class other); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java index 25c785b542e8..610e92a0ea6b 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ShardSpecLookup.java @@ -23,5 +23,9 @@ public interface ShardSpecLookup { + /** + * Returns a {@link ShardSpec} for the given timestamp and the inputRow. + * The timestamp must be bucketed using {@code GranularitySpec#getQueryGranularity}. + */ ShardSpec getShardSpec(long timestamp, InputRow row); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java index 22ca97d3fe19..e2ebd295c356 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionPartialShardSpec.java @@ -87,15 +87,19 @@ public int getNumBuckets() @Override public ShardSpec complete(ObjectMapper objectMapper, @Nullable ShardSpec specOfPreviousMaxPartitionId) { - final int partitionId; - if (specOfPreviousMaxPartitionId != null) { - assert specOfPreviousMaxPartitionId instanceof SingleDimensionShardSpec; - final SingleDimensionShardSpec prevSpec = (SingleDimensionShardSpec) specOfPreviousMaxPartitionId; - partitionId = prevSpec.getPartitionNum() + 1; - } else { - partitionId = 0; - } - return complete(objectMapper, partitionId); + // The shardSpec is created by the Overlord. + // For batch tasks, this code can be executed only with segment locking (forceTimeChunkLock = false). + // In this mode, you can have 2 or more tasks concurrently ingesting into the same time chunk of + // the same datasource. Since there is no restriction for those tasks in segment allocation, the + // allocated IDs for each task can interleave. As a result, the core partition set cannot be + // represented as a range. We always set 0 for the core partition set size if this is an initial segment. + return new SingleDimensionShardSpec( + partitionDimension, + start, + end, + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getPartitionNum() + 1, + specOfPreviousMaxPartitionId == null ? 0 : specOfPreviousMaxPartitionId.getNumCorePartitions() + ); } @Override @@ -106,7 +110,8 @@ public ShardSpec complete(ObjectMapper objectMapper, int partitionId) partitionDimension, start, end, - partitionId + partitionId, + 0 ); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java index 9db390c462fe..1a00534d6ce8 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/SingleDimensionShardSpec.java @@ -38,12 +38,15 @@ */ public class SingleDimensionShardSpec implements ShardSpec { + public static final int UNKNOWN_NUM_CORE_PARTITIONS = -1; + private final String dimension; @Nullable private final String start; @Nullable private final String end; private final int partitionNum; + private final int numCorePartitions; /** * @param dimension partition dimension @@ -56,7 +59,8 @@ public SingleDimensionShardSpec( @JsonProperty("dimension") String dimension, @JsonProperty("start") @Nullable String start, @JsonProperty("end") @Nullable String end, - @JsonProperty("partitionNum") int partitionNum + @JsonProperty("partitionNum") int partitionNum, + @JsonProperty("numCorePartitions") @Nullable Integer numCorePartitions // nullable for backward compatibility ) { Preconditions.checkArgument(partitionNum >= 0, "partitionNum >= 0"); @@ -64,6 +68,18 @@ public SingleDimensionShardSpec( this.start = start; this.end = end; this.partitionNum = partitionNum; + this.numCorePartitions = numCorePartitions == null ? UNKNOWN_NUM_CORE_PARTITIONS : numCorePartitions; + } + + public SingleDimensionShardSpec withNumCorePartitions(int numCorePartitions) + { + return new SingleDimensionShardSpec( + dimension, + start, + end, + partitionNum, + numCorePartitions + ); } @JsonProperty("dimension") @@ -94,7 +110,19 @@ public int getPartitionNum() } @Override - public ShardSpecLookup getLookup(final List shardSpecs) + @JsonProperty + public int getNumCorePartitions() + { + return numCorePartitions; + } + + @Override + public ShardSpecLookup getLookup(final List shardSpecs) + { + return createLookup(shardSpecs); + } + + static ShardSpecLookup createLookup(List shardSpecs) { return (long timestamp, InputRow row) -> { for (ShardSpec spec : shardSpecs) { @@ -146,22 +174,20 @@ public boolean isCompatible(Class other) @Override public PartitionChunk createChunk(T obj) { - return new StringPartitionChunk(start, end, partitionNum, obj); + if (numCorePartitions == UNKNOWN_NUM_CORE_PARTITIONS) { + return new StringPartitionChunk<>(start, end, partitionNum, obj); + } else { + return new NumberedPartitionChunk<>(partitionNum, numCorePartitions, obj); + } } @Override public boolean isInChunk(long timestamp, InputRow inputRow) { - final List values = inputRow.getDimension(dimension); - - if (values == null || values.size() != 1) { - return checkValue(null); - } else { - return checkValue(values.get(0)); - } + return isInChunk(dimension, start, end, inputRow); } - private boolean checkValue(String value) + private static boolean checkValue(@Nullable String start, @Nullable String end, String value) { if (value == null) { return start == null; @@ -175,15 +201,20 @@ private boolean checkValue(String value) (end == null || value.compareTo(end) < 0); } - @Override - public String toString() + public static boolean isInChunk( + String dimension, + @Nullable String start, + @Nullable String end, + InputRow inputRow + ) { - return "SingleDimensionShardSpec{" + - "dimension='" + dimension + '\'' + - ", start='" + start + '\'' + - ", end='" + end + '\'' + - ", partitionNum=" + partitionNum + - '}'; + final List values = inputRow.getDimension(dimension); + + if (values == null || values.size() != 1) { + return checkValue(start, end, null); + } else { + return checkValue(start, end, values.get(0)); + } } @Override @@ -195,16 +226,29 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - SingleDimensionShardSpec that = (SingleDimensionShardSpec) o; - return partitionNum == that.partitionNum && - Objects.equals(dimension, that.dimension) && - Objects.equals(start, that.start) && - Objects.equals(end, that.end); + SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) o; + return partitionNum == shardSpec.partitionNum && + numCorePartitions == shardSpec.numCorePartitions && + Objects.equals(dimension, shardSpec.dimension) && + Objects.equals(start, shardSpec.start) && + Objects.equals(end, shardSpec.end); } @Override public int hashCode() { - return Objects.hash(dimension, start, end, partitionNum); + return Objects.hash(dimension, start, end, partitionNum, numCorePartitions); + } + + @Override + public String toString() + { + return "SingleDimensionShardSpec{" + + "dimension='" + dimension + '\'' + + ", start='" + start + '\'' + + ", end='" + end + '\'' + + ", partitionNum=" + partitionNum + + ", numCorePartitions=" + numCorePartitions + + '}'; } } diff --git a/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java b/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java index 9efb9b8341ee..cc5b82ba26e0 100644 --- a/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java +++ b/core/src/test/java/org/apache/druid/collections/BlockingPoolTest.java @@ -20,6 +20,7 @@ package org.apache.druid.collections; import com.google.common.base.Suppliers; +import com.google.common.collect.Iterables; import org.apache.druid.java.util.common.concurrent.Execs; import org.junit.After; import org.junit.Assert; @@ -66,7 +67,7 @@ public void testTakeFromEmptyPool() { expectedException.expect(IllegalStateException.class); expectedException.expectMessage("Pool was initialized with limit = 0, there are no objects to take."); - emptyPool.take(0); + emptyPool.takeBatch(1, 0); } @Test @@ -80,7 +81,7 @@ public void testDrainFromEmptyPool() @Test(timeout = 60_000L) public void testTake() { - final ReferenceCountingResourceHolder holder = pool.take(100); + final ReferenceCountingResourceHolder holder = Iterables.getOnlyElement(pool.takeBatch(1, 100), null); Assert.assertNotNull(holder); Assert.assertEquals(9, pool.getPoolSize()); holder.close(); @@ -91,7 +92,7 @@ public void testTake() public void testTakeTimeout() { final List> batchHolder = pool.takeBatch(10, 100L); - final ReferenceCountingResourceHolder holder = pool.take(100); + final ReferenceCountingResourceHolder holder = Iterables.getOnlyElement(pool.takeBatch(1, 100), null); Assert.assertNull(holder); batchHolder.forEach(ReferenceCountingResourceHolder::close); } @@ -147,7 +148,7 @@ public void testConcurrentTake() throws ExecutionException, InterruptedException () -> { List> result = new ArrayList<>(); for (int i = 0; i < limit1; i++) { - result.add(pool.take(10)); + result.add(Iterables.getOnlyElement(pool.takeBatch(1, 10), null)); } return result; } @@ -156,7 +157,7 @@ public void testConcurrentTake() throws ExecutionException, InterruptedException () -> { List> result = new ArrayList<>(); for (int i = 0; i < limit2; i++) { - result.add(pool.take(10)); + result.add(Iterables.getOnlyElement(pool.takeBatch(1, 10), null)); } return result; } diff --git a/core/src/test/java/org/apache/druid/common/guava/CombiningSequenceTest.java b/core/src/test/java/org/apache/druid/common/guava/CombiningSequenceTest.java index 9f64d6732863..b8872f08d385 100644 --- a/core/src/test/java/org/apache/druid/common/guava/CombiningSequenceTest.java +++ b/core/src/test/java/org/apache/druid/common/guava/CombiningSequenceTest.java @@ -20,17 +20,21 @@ package org.apache.druid.common.guava; import com.google.common.base.Predicate; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.java.util.common.guava.ExplodingSequence; 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.Yielder; import org.apache.druid.java.util.common.guava.YieldingAccumulator; +import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -39,6 +43,7 @@ import java.util.Arrays; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.concurrent.CountDownLatch; @@ -194,6 +199,40 @@ public void testNothing() throws Exception testCombining(Collections.emptyList(), Collections.emptyList()); } + @Test + public void testExplodingSequence() + { + final ExplodingSequence bomb = + new ExplodingSequence<>(Sequences.simple(ImmutableList.of(1, 2, 2)), false, true); + + final CombiningSequence combiningSequence = + CombiningSequence.create(bomb, Comparator.naturalOrder(), (a, b) -> a); + + try { + combiningSequence.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Integer accumulate(Integer accumulated, Integer in) + { + if (in > 1) { + throw new RuntimeException("boom"); + } + + return in; + } + } + ); + Assert.fail("Expected exception"); + } + catch (Exception e) { + Assert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("boom"))); + } + + Assert.assertEquals("Closes resources", 1, bomb.getCloseCount()); + } + private void testCombining(List> pairs, List> expected) throws Exception { diff --git a/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java b/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java index 1d4cb6dbade9..bd5c0b2ba6c1 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/StringUtilsTest.java @@ -133,13 +133,6 @@ public void fromUtf8ByteBufferDirect() Assert.assertEquals("abcd", StringUtils.fromUtf8(bytes)); } - @Test - public void testCharsetShowsUpAsDeprecated() - { - // Not actually a runnable test, just checking the IDE - Assert.assertNotNull(StringUtils.UTF8_CHARSET); - } - @SuppressWarnings("MalformedFormatString") @Test public void testNonStrictFormat() @@ -211,39 +204,51 @@ public void testRepeat() @Test public void testLpad() { - String s1 = StringUtils.lpad("abc", 7, "de"); - Assert.assertEquals(s1, "dedeabc"); + String lpad = StringUtils.lpad("abc", 7, "de"); + Assert.assertEquals("dedeabc", lpad); - String s2 = StringUtils.lpad("abc", 6, "de"); - Assert.assertEquals(s2, "dedabc"); + lpad = StringUtils.lpad("abc", 6, "de"); + Assert.assertEquals("dedabc", lpad); - String s3 = StringUtils.lpad("abc", 2, "de"); - Assert.assertEquals(s3, "ab"); + lpad = StringUtils.lpad("abc", 2, "de"); + Assert.assertEquals("ab", lpad); - String s4 = StringUtils.lpad("abc", 0, "de"); - Assert.assertEquals(s4, ""); + lpad = StringUtils.lpad("abc", 0, "de"); + Assert.assertEquals("", lpad); - String s5 = StringUtils.lpad("abc", -1, "de"); - Assert.assertEquals(s5, null); + lpad = StringUtils.lpad("abc", -1, "de"); + Assert.assertEquals("", lpad); + + lpad = StringUtils.lpad("abc", 10, ""); + Assert.assertEquals("abc", lpad); + + lpad = StringUtils.lpad("abc", 1, ""); + Assert.assertEquals("a", lpad); } @Test public void testRpad() { - String s1 = StringUtils.rpad("abc", 7, "de"); - Assert.assertEquals(s1, "abcdede"); + String rpad = StringUtils.rpad("abc", 7, "de"); + Assert.assertEquals("abcdede", rpad); + + rpad = StringUtils.rpad("abc", 6, "de"); + Assert.assertEquals("abcded", rpad); + + rpad = StringUtils.rpad("abc", 2, "de"); + Assert.assertEquals("ab", rpad); - String s2 = StringUtils.rpad("abc", 6, "de"); - Assert.assertEquals(s2, "abcded"); + rpad = StringUtils.rpad("abc", 0, "de"); + Assert.assertEquals("", rpad); - String s3 = StringUtils.rpad("abc", 2, "de"); - Assert.assertEquals(s3, "ab"); + rpad = StringUtils.rpad("abc", -1, "de"); + Assert.assertEquals("", rpad); - String s4 = StringUtils.rpad("abc", 0, "de"); - Assert.assertEquals(s4, ""); + rpad = StringUtils.rpad("abc", 10, ""); + Assert.assertEquals("abc", rpad); - String s5 = StringUtils.rpad("abc", -1, "de"); - Assert.assertEquals(s5, null); + rpad = StringUtils.rpad("abc", 1, ""); + Assert.assertEquals("a", rpad); } @Test diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java index 61af63869487..f7817c75293a 100644 --- a/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/MergeSequenceTest.java @@ -19,10 +19,13 @@ package org.apache.druid.java.util.common.guava; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; -import junit.framework.Assert; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; import java.util.ArrayList; import java.util.Arrays; @@ -41,7 +44,10 @@ public void testSanity() throws Exception TestSequence.create(4, 6, 8) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + (Sequence) Sequences.simple(testSeqs) + ); SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); for (TestSequence sequence : testSeqs) { @@ -58,7 +64,10 @@ public void testWorksWhenBeginningOutOfOrder() throws Exception TestSequence.create(4, 6, 8) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + (Sequence) Sequences.simple(testSeqs) + ); SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); for (TestSequence sequence : testSeqs) { @@ -76,7 +85,10 @@ public void testMergeEmpties() throws Exception TestSequence.create(4, 6, 8) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + (Sequence) Sequences.simple(testSeqs) + ); SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); for (TestSequence sequence : testSeqs) { @@ -94,7 +106,10 @@ public void testMergeEmpties1() throws Exception TestSequence.create(4, 6, 8) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + (Sequence) Sequences.simple(testSeqs) + ); SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); for (TestSequence sequence : testSeqs) { @@ -113,7 +128,10 @@ public void testMergeEmpties2() throws Exception TestSequence.create() ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + (Sequence) Sequences.simple(testSeqs) + ); SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 8, 9)); for (TestSequence sequence : testSeqs) { @@ -130,7 +148,10 @@ public void testScrewsUpOnOutOfOrder() throws Exception TestSequence.create(4, 6) ); - MergeSequence seq = new MergeSequence<>(Ordering.natural(), (Sequence) Sequences.simple(testSeqs)); + MergeSequence seq = new MergeSequence<>( + Ordering.natural(), + (Sequence) Sequences.simple(testSeqs) + ); SequenceTestHelper.testAll(seq, Arrays.asList(1, 2, 3, 4, 5, 4, 6, 7, 8, 9)); for (TestSequence sequence : testSeqs) { @@ -169,4 +190,116 @@ public void testMergeOne() throws Exception SequenceTestHelper.testAll(mergeOne, Collections.singletonList(1)); } + @Test + public void testTwoExplodingOnGetSequences() + { + final ExplodingSequence bomb1 = + new ExplodingSequence<>(Sequences.simple(ImmutableList.of(1, 2, 2)), true, false); + final ExplodingSequence bomb2 = + new ExplodingSequence<>(Sequences.simple(ImmutableList.of(1, 2, 2)), true, false); + + final MergeSequence mergeSequence = + new MergeSequence<>( + Ordering.natural(), + Sequences.simple(ImmutableList.of(bomb1, bomb2)) + ); + + try { + mergeSequence.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Integer accumulate(Integer accumulated, Integer in) + { + return in; + } + } + ); + Assert.fail("Expected exception"); + } + catch (Exception e) { + Assert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("get"))); + } + + Assert.assertEquals("Closes resources (1)", 1, bomb1.getCloseCount()); + Assert.assertEquals("Closes resources (2)", 1, bomb2.getCloseCount()); + } + + @Test + public void testTwoExplodingOnCloseSequences() + { + final ExplodingSequence bomb1 = + new ExplodingSequence<>(Sequences.simple(ImmutableList.of(1, 2, 2)), false, true); + final ExplodingSequence bomb2 = + new ExplodingSequence<>(Sequences.simple(ImmutableList.of(1, 2, 2)), false, true); + + final MergeSequence mergeSequence = + new MergeSequence<>( + Ordering.natural(), + Sequences.simple(ImmutableList.of(bomb1, bomb2)) + ); + + try { + mergeSequence.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Integer accumulate(Integer accumulated, Integer in) + { + if (in > 1) { + throw new RuntimeException("boom"); + } + + return in; + } + } + ); + Assert.fail("Expected exception"); + } + catch (Exception e) { + Assert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("boom"))); + } + + Assert.assertEquals("Closes resources (1)", 1, bomb1.getCloseCount()); + Assert.assertEquals("Closes resources (2)", 1, bomb2.getCloseCount()); + } + + @Test + public void testOneEmptyOneExplodingSequence() + { + final ExplodingSequence bomb = + new ExplodingSequence<>(Sequences.simple(ImmutableList.of(1, 2, 2)), false, true); + + final MergeSequence mergeSequence = + new MergeSequence<>( + Ordering.natural(), + Sequences.simple(ImmutableList.of(Sequences.empty(), bomb)) + ); + + try { + mergeSequence.toYielder( + null, + new YieldingAccumulator() + { + @Override + public Integer accumulate(Integer accumulated, Integer in) + { + if (in > 1) { + throw new RuntimeException("boom"); + } + + return in; + } + } + ); + Assert.fail("Expected exception"); + } + catch (Exception e) { + Assert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("boom"))); + } + + Assert.assertEquals("Closes resources", 1, bomb.getCloseCount()); + } } diff --git a/core/src/test/java/org/apache/druid/java/util/common/guava/YieldingSequenceBaseTest.java b/core/src/test/java/org/apache/druid/java/util/common/guava/YieldingSequenceBaseTest.java new file mode 100644 index 000000000000..19e396bc064c --- /dev/null +++ b/core/src/test/java/org/apache/druid/java/util/common/guava/YieldingSequenceBaseTest.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.java.util.common.guava; + +import com.google.common.collect.ImmutableList; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableCauseMatcher; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +import java.util.ArrayList; + +public class YieldingSequenceBaseTest +{ + @Test + public void testAccumulate() + { + final ExplodingSequence sequence = new ExplodingSequence<>( + Sequences.simple(ImmutableList.of(1, 2, 3)), + false, + false + ); + + Assert.assertEquals(ImmutableList.of(1, 2, 3), sequence.accumulate(new ArrayList<>(), Accumulators.list())); + Assert.assertEquals("Closes resources", 1, sequence.getCloseCount()); + } + + @Test + public void testExceptionDuringGet() + { + final ExplodingSequence sequence = new ExplodingSequence<>( + Sequences.simple(ImmutableList.of(1, 2, 3)), + true, + false + ); + + try { + sequence.accumulate(new ArrayList<>(), Accumulators.list()); + Assert.fail("Expected exception"); + } + catch (Exception e) { + Assert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("get"))); + } + + Assert.assertEquals("Closes resources", 1, sequence.getCloseCount()); + } + + @Test + public void testExceptionDuringClose() + { + final ExplodingSequence sequence = new ExplodingSequence<>( + Sequences.simple(ImmutableList.of(1, 2, 3)), + false, + true + ); + + try { + sequence.accumulate(new ArrayList<>(), Accumulators.list()); + Assert.fail("Expected exception"); + } + catch (Exception e) { + Assert.assertThat( + e, + + // Wrapped one level deep because it's an IOException + ThrowableCauseMatcher.hasCause(ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("close"))) + ); + } + + Assert.assertEquals("Closes resources", 1, sequence.getCloseCount()); + } + + @Test + public void testExceptionDuringGetAndClose() + { + final ExplodingSequence sequence = new ExplodingSequence<>( + Sequences.simple(ImmutableList.of(1, 2, 3)), + true, + true + ); + + try { + sequence.accumulate(new ArrayList<>(), Accumulators.list()); + Assert.fail("Expected exception"); + } + catch (Exception e) { + Assert.assertThat(e, ThrowableMessageMatcher.hasMessage(CoreMatchers.equalTo("get"))); + } + + Assert.assertEquals("Closes resources", 1, sequence.getCloseCount()); + } +} diff --git a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java index 2fe52490c400..bd755ba7e0ec 100644 --- a/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java +++ b/core/src/test/java/org/apache/druid/math/expr/FunctionTest.java @@ -20,13 +20,19 @@ package org.apache.druid.math.expr; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import javax.annotation.Nullable; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.util.Locale; +import java.util.Set; public class FunctionTest extends InitializedNullHandlingTest { @@ -35,13 +41,23 @@ public class FunctionTest extends InitializedNullHandlingTest @Before public void setup() { - ImmutableMap.Builder builder = ImmutableMap.builder(); - builder.put("x", "foo"); - builder.put("y", 2); - builder.put("z", 3.1); - builder.put("a", new String[] {"foo", "bar", "baz", "foobar"}); - builder.put("b", new Long[] {1L, 2L, 3L, 4L, 5L}); - builder.put("c", new Double[] {3.1, 4.2, 5.3}); + ImmutableMap.Builder builder = ImmutableMap.builder() + .put("x", "foo") + .put("y", 2) + .put("z", 3.1) + .put("d", 34.56D) + .put("maxLong", Long.MAX_VALUE) + .put("minLong", Long.MIN_VALUE) + .put("f", 12.34F) + .put("nan", Double.NaN) + .put("inf", Double.POSITIVE_INFINITY) + .put("-inf", Double.NEGATIVE_INFINITY) + .put("o", 0) + .put("od", 0D) + .put("of", 0F) + .put("a", new String[] {"foo", "bar", "baz", "foobar"}) + .put("b", new Long[] {1L, 2L, 3L, 4L, 5L}) + .put("c", new Double[] {3.1, 4.2, 5.3}); bindings = Parser.withMap(builder.build()); } @@ -146,9 +162,23 @@ public void testLpad() assertExpr("lpad(x, 5, 'ab')", "abfoo"); assertExpr("lpad(x, 4, 'ab')", "afoo"); assertExpr("lpad(x, 2, 'ab')", "fo"); - assertArrayExpr("lpad(x, 0, 'ab')", null); - assertArrayExpr("lpad(x, 5, null)", null); - assertArrayExpr("lpad(null, 5, x)", null); + assertExpr("lpad(x, -1, 'ab')", NullHandling.replaceWithDefault() ? null : ""); + assertExpr("lpad(null, 5, 'ab')", null); + assertExpr("lpad(x, 2, '')", NullHandling.replaceWithDefault() ? null : "fo"); + assertExpr("lpad(x, 6, '')", NullHandling.replaceWithDefault() ? null : "foo"); + assertExpr("lpad('', 3, '*')", NullHandling.replaceWithDefault() ? null : "***"); + assertExpr("lpad(x, 2, null)", null); + assertExpr("lpad(a, 4, '*')", "[foo"); + assertExpr("lpad(a, 2, '*')", "[f"); + assertExpr("lpad(a, 2, '')", NullHandling.replaceWithDefault() ? null : "[f"); + assertExpr("lpad(b, 4, '*')", "[1, "); + assertExpr("lpad(b, 2, '')", NullHandling.replaceWithDefault() ? null : "[1"); + assertExpr("lpad(b, 2, null)", null); + assertExpr("lpad(x, 5, x)", "fofoo"); + assertExpr("lpad(x, 5, y)", "22foo"); + assertExpr("lpad(x, 5, z)", "3.foo"); + assertExpr("lpad(y, 5, x)", "foof2"); + assertExpr("lpad(z, 5, y)", "223.1"); } @Test @@ -157,9 +187,22 @@ public void testRpad() assertExpr("rpad(x, 5, 'ab')", "fooab"); assertExpr("rpad(x, 4, 'ab')", "fooa"); assertExpr("rpad(x, 2, 'ab')", "fo"); - assertArrayExpr("rpad(x, 0, 'ab')", null); - assertArrayExpr("rpad(x, 5, null)", null); - assertArrayExpr("rpad(null, 5, x)", null); + assertExpr("rpad(x, -1, 'ab')", NullHandling.replaceWithDefault() ? null : ""); + assertExpr("rpad(null, 5, 'ab')", null); + assertExpr("rpad(x, 2, '')", NullHandling.replaceWithDefault() ? null : "fo"); + assertExpr("rpad(x, 6, '')", NullHandling.replaceWithDefault() ? null : "foo"); + assertExpr("rpad('', 3, '*')", NullHandling.replaceWithDefault() ? null : "***"); + assertExpr("rpad(x, 2, null)", null); + assertExpr("rpad(a, 2, '*')", "[f"); + assertExpr("rpad(a, 2, '')", NullHandling.replaceWithDefault() ? null : "[f"); + assertExpr("rpad(b, 4, '*')", "[1, "); + assertExpr("rpad(b, 2, '')", NullHandling.replaceWithDefault() ? null : "[1"); + assertExpr("rpad(b, 2, null)", null); + assertExpr("rpad(x, 5, x)", "foofo"); + assertExpr("rpad(x, 5, y)", "foo22"); + assertExpr("rpad(x, 5, z)", "foo3."); + assertExpr("rpad(y, 5, x)", "2foof"); + assertExpr("rpad(z, 5, y)", "3.122"); } @Test @@ -293,6 +336,133 @@ public void testArrayPrepend() assertArrayExpr("array_prepend(1, [])", new Double[]{1.0}); } + @Test + public void testRoundWithNonNumericValuesShouldReturn0() + { + assertExpr("round(nan)", 0D); + assertExpr("round(nan, 5)", 0D); + //CHECKSTYLE.OFF: Regexp + assertExpr("round(inf)", Double.MAX_VALUE); + assertExpr("round(inf, 4)", Double.MAX_VALUE); + assertExpr("round(-inf)", -1 * Double.MAX_VALUE); + assertExpr("round(-inf, 3)", -1 * Double.MAX_VALUE); + assertExpr("round(-inf, -5)", -1 * Double.MAX_VALUE); + //CHECKSTYLE.ON: Regexp + + // Calculations that result in non numeric numbers + assertExpr("round(0/od)", 0D); + assertExpr("round(od/od)", 0D); + //CHECKSTYLE.OFF: Regexp + assertExpr("round(1/od)", Double.MAX_VALUE); + assertExpr("round(-1/od)", -1 * Double.MAX_VALUE); + //CHECKSTYLE.ON: Regexp + + assertExpr("round(0/of)", 0D); + assertExpr("round(of/of)", 0D); + //CHECKSTYLE.OFF: Regexp + assertExpr("round(1/of)", Double.MAX_VALUE); + assertExpr("round(-1/of)", -1 * Double.MAX_VALUE); + //CHECKSTYLE.ON: Regexp + } + + @Test + public void testRoundWithLong() + { + assertExpr("round(y)", 2L); + assertExpr("round(y, 2)", 2L); + assertExpr("round(y, -1)", 0L); + } + + @Test + public void testRoundWithDouble() + { + assertExpr("round(d)", 35D); + assertExpr("round(d, 2)", 34.56D); + assertExpr("round(d, y)", 34.56D); + assertExpr("round(d, 1)", 34.6D); + assertExpr("round(d, -1)", 30D); + } + + @Test + public void testRoundWithFloat() + { + assertExpr("round(f)", 12D); + assertExpr("round(f, 2)", 12.34D); + assertExpr("round(f, y)", 12.34D); + assertExpr("round(f, 1)", 12.3D); + assertExpr("round(f, -1)", 10D); + } + + @Test + public void testRoundWithExtremeNumbers() + { + assertExpr("round(maxLong)", BigDecimal.valueOf(Long.MAX_VALUE).setScale(0, RoundingMode.HALF_UP).longValue()); + assertExpr("round(minLong)", BigDecimal.valueOf(Long.MIN_VALUE).setScale(0, RoundingMode.HALF_UP).longValue()); + // overflow + assertExpr("round(maxLong + 1, 1)", BigDecimal.valueOf(Long.MIN_VALUE).setScale(1, RoundingMode.HALF_UP).longValue()); + // underflow + assertExpr("round(minLong - 1, -2)", BigDecimal.valueOf(Long.MAX_VALUE).setScale(-2, RoundingMode.HALF_UP).longValue()); + + assertExpr("round(CAST(maxLong, 'DOUBLE') + 1, 1)", BigDecimal.valueOf(((double) Long.MAX_VALUE) + 1).setScale(1, RoundingMode.HALF_UP).doubleValue()); + assertExpr("round(CAST(minLong, 'DOUBLE') - 1, -2)", BigDecimal.valueOf(((double) Long.MIN_VALUE) - 1).setScale(-2, RoundingMode.HALF_UP).doubleValue()); + } + + @Test + public void testRoundWithInvalidFirstArgument() + { + Set> invalidArguments = ImmutableSet.of( + Pair.of("b", "LONG_ARRAY"), + Pair.of("x", "STRING"), + Pair.of("c", "DOUBLE_ARRAY"), + Pair.of("a", "STRING_ARRAY") + + ); + for (Pair argAndType : invalidArguments) { + try { + assertExpr(String.format(Locale.ENGLISH, "round(%s)", argAndType.lhs), null); + Assert.fail("Did not throw IllegalArgumentException"); + } + catch (IllegalArgumentException e) { + Assert.assertEquals( + String.format( + Locale.ENGLISH, + "The first argument to the function[round] should be integer or double type but got the type: %s", + argAndType.rhs + ), + e.getMessage() + ); + } + } + } + + @Test + public void testRoundWithInvalidSecondArgument() + { + Set> invalidArguments = ImmutableSet.of( + Pair.of("1.2", "DOUBLE"), + Pair.of("x", "STRING"), + Pair.of("a", "STRING_ARRAY"), + Pair.of("c", "DOUBLE_ARRAY") + + ); + for (Pair argAndType : invalidArguments) { + try { + assertExpr(String.format(Locale.ENGLISH, "round(d, %s)", argAndType.lhs), null); + Assert.fail("Did not throw IllegalArgumentException"); + } + catch (IllegalArgumentException e) { + Assert.assertEquals( + String.format( + Locale.ENGLISH, + "The second argument to the function[round] should be integer type but got the type: %s", + argAndType.rhs + ), + e.getMessage() + ); + } + } + } + @Test public void testGreatest() { diff --git a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index 6d0af3f2892e..c2b0b76e8881 100644 --- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -75,7 +75,13 @@ public int getPartitionNum() } @Override - public ShardSpecLookup getLookup(List shardSpecs) + public int getNumCorePartitions() + { + return 0; + } + + @Override + public ShardSpecLookup getLookup(List shardSpecs) { return null; } diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.java new file mode 100644 index 000000000000..2c052d51c228 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingHashBasedNumberedShardSpecTest.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.timeline.partition; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +public class BuildingHashBasedNumberedShardSpecTest +{ + private final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + + @Test + public void testConvert() + { + Assert.assertEquals( + new HashBasedNumberedShardSpec(5, 10, 5, 12, ImmutableList.of("dim"), mapper), + new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), mapper).convert(10) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new BuildingHashBasedNumberedShardSpec(5, 5, 12, ImmutableList.of("dim"), mapper) + .createChunk("test") + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + mapper.registerSubtypes( + new NamedType(BuildingHashBasedNumberedShardSpec.class, BuildingHashBasedNumberedShardSpec.TYPE) + ); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + final BuildingHashBasedNumberedShardSpec original = new BuildingHashBasedNumberedShardSpec( + 3, + 5, + 12, + ImmutableList.of("dim"), + mapper + ); + final String json = mapper.writeValueAsString(original); + final BuildingHashBasedNumberedShardSpec fromJson = (BuildingHashBasedNumberedShardSpec) mapper.readValue( + json, + ShardSpec.class + ); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(BuildingHashBasedNumberedShardSpec.class) + .withIgnoredFields("jsonMapper") + .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper()) + .usingGetClass() + .verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java new file mode 100644 index 000000000000..b608d4cda730 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingNumberedShardSpecTest.java @@ -0,0 +1,64 @@ +/* + * 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.timeline.partition; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +public class BuildingNumberedShardSpecTest +{ + @Test + public void testConvert() + { + Assert.assertEquals(new NumberedShardSpec(5, 10), new BuildingNumberedShardSpec(5).convert(10)); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new BuildingNumberedShardSpec(5).createChunk("test") + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.registerSubtypes(new NamedType(BuildingNumberedShardSpec.class, BuildingNumberedShardSpec.TYPE)); + final BuildingNumberedShardSpec original = new BuildingNumberedShardSpec(5); + final String json = mapper.writeValueAsString(original); + final BuildingNumberedShardSpec fromJson = (BuildingNumberedShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(BuildingNumberedShardSpec.class).usingGetClass().verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpecTest.java new file mode 100644 index 000000000000..d70a42ff5bae --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/BuildingSingleDimensionShardSpecTest.java @@ -0,0 +1,72 @@ +/* + * 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.timeline.partition; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +public class BuildingSingleDimensionShardSpecTest +{ + @Test + public void testConvert() + { + Assert.assertEquals( + new SingleDimensionShardSpec("dim", "start", "end", 5, 10), + new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5).convert(10) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5).createChunk("test") + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + mapper.registerSubtypes( + new NamedType(BuildingSingleDimensionShardSpec.class, BuildingSingleDimensionShardSpec.TYPE) + ); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + final BuildingSingleDimensionShardSpec original = new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5); + final String json = mapper.writeValueAsString(original); + final BuildingSingleDimensionShardSpec fromJson = (BuildingSingleDimensionShardSpec) mapper.readValue( + json, + ShardSpec.class + ); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(BuildingSingleDimensionShardSpec.class).usingGetClass().verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java index 9b2c664679f3..551992bdb2d6 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedPartialShardSpecTest.java @@ -46,6 +46,7 @@ public void testSerde() throws IOException { final HashBasedNumberedPartialShardSpec expected = new HashBasedNumberedPartialShardSpec( ImmutableList.of("dim1", "dim2"), + 1, 3 ); final byte[] json = MAPPER.writeValueAsBytes(expected); @@ -61,14 +62,16 @@ public void testJsonPropertyNames() throws IOException { final HashBasedNumberedPartialShardSpec expected = new HashBasedNumberedPartialShardSpec( ImmutableList.of("dim1", "dim2"), + 1, 3 ); final byte[] json = MAPPER.writeValueAsBytes(expected); //noinspection unchecked final Map map = MAPPER.readValue(json, Map.class); - Assert.assertEquals(3, map.size()); + Assert.assertEquals(4, map.size()); Assert.assertEquals(HashBasedNumberedPartialShardSpec.TYPE, map.get("type")); Assert.assertEquals(expected.getPartitionDimensions(), map.get("partitionDimensions")); + Assert.assertEquals(expected.getBucketId(), map.get("bucketId")); Assert.assertEquals(expected.getNumBuckets(), map.get("numPartitions")); } } diff --git a/core/src/test/java/org/apache/druid/timeline/partition/HashBucketShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/HashBucketShardSpecTest.java new file mode 100644 index 000000000000..df2207b798f4 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/HashBucketShardSpecTest.java @@ -0,0 +1,121 @@ +/* + * 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.timeline.partition; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class HashBucketShardSpecTest +{ + private final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + + @Test + public void testConvert() + { + Assert.assertEquals( + new BuildingHashBasedNumberedShardSpec(3, 5, 12, ImmutableList.of("dim"), mapper), + new HashBucketShardSpec(5, 12, ImmutableList.of("dim"), mapper).convert(3) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(5, 0, "test"), + new HashBucketShardSpec(5, 12, ImmutableList.of("dim"), mapper).createChunk("test") + ); + } + + @Test + public void testShardSpecLookup() + { + final List shardSpecs = ImmutableList.of( + new HashBucketShardSpec(0, 3, ImmutableList.of("dim"), mapper), + new HashBucketShardSpec(1, 3, ImmutableList.of("dim"), mapper), + new HashBucketShardSpec(2, 3, ImmutableList.of("dim"), mapper) + ); + final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); + final long currentTime = DateTimes.nowUtc().getMillis(); + Assert.assertEquals( + shardSpecs.get(1), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "1", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(2), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "2", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(0), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "3", "time", currentTime) + ) + ) + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + mapper.registerSubtypes(new NamedType(HashBucketShardSpec.class, HashBucketShardSpec.TYPE)); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + + final HashBucketShardSpec original = new HashBucketShardSpec(5, 12, ImmutableList.of("dim"), mapper); + final String json = mapper.writeValueAsString(original); + final HashBucketShardSpec fromJson = (HashBucketShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(HashBucketShardSpec.class) + .withIgnoredFields("jsonMapper") + .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper()) + .usingGetClass() + .verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.java new file mode 100644 index 000000000000..c6d7935b5699 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/NumberedOverwriteShardSpecTest.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.timeline.partition; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Assert; +import org.junit.Test; + +public class NumberedOverwriteShardSpecTest +{ + @Test + public void testEquals() + { + EqualsVerifier.forClass(NumberedOverwriteShardSpec.class).usingGetClass().verify(); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + mapper.registerSubtypes(new NamedType(NumberedOverwriteShardSpec.class, NumberedOverwriteShardSpec.TYPE)); + final NumberedOverwriteShardSpec original = new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 2, + 0, + 10, + (short) 1, + (short) 3 + ); + final String json = mapper.writeValueAsString(original); + final NumberedOverwriteShardSpec fromJson = (NumberedOverwriteShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java b/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.java new file mode 100644 index 000000000000..38b9a47854f2 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/PartitionHolderCompletenessTest.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.timeline.partition; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import org.apache.druid.java.util.common.StringUtils; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.List; + +@RunWith(Parameterized.class) +public class PartitionHolderCompletenessTest +{ + @Parameterized.Parameters(name = "{1}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{ + ImmutableList.of( + new NumberedShardSpec(0, 3), + new NumberedShardSpec(1, 3), + new NumberedShardSpec(2, 3) + ), + NumberedShardSpec.class.getSimpleName() + }, + new Object[]{ + // Simulate empty hash buckets + ImmutableList.of( + new HashBasedNumberedShardSpec(0, 3, 0, 5, null, new ObjectMapper()), + new HashBasedNumberedShardSpec(1, 3, 2, 5, null, new ObjectMapper()), + new HashBasedNumberedShardSpec(2, 3, 3, 5, null, new ObjectMapper()) + ), + HashBasedNumberedShardSpec.class.getSimpleName() + }, + new Object[]{ + // Simulate empty range buckets + ImmutableList.of( + new SingleDimensionShardSpec("dim", null, "aaa", 0, 3), + new SingleDimensionShardSpec("dim", "bbb", "fff", 1, 3), + new SingleDimensionShardSpec("dim", "ttt", "zzz", 2, 3) + ), + StringUtils.format( + "%s with empty buckets", + SingleDimensionShardSpec.class.getSimpleName() + ) + }, + new Object[]{ + // Simulate old format segments with missing numCorePartitions + ImmutableList.of( + new SingleDimensionShardSpec("dim", null, "bbb", 0, null), + new SingleDimensionShardSpec("dim", "bbb", "fff", 1, null), + new SingleDimensionShardSpec("dim", "fff", null, 2, null) + ), + StringUtils.format( + "%s with missing numCorePartitions", + SingleDimensionShardSpec.class.getSimpleName() + ) + } + ); + } + + private final List shardSpecs; + + public PartitionHolderCompletenessTest(List shardSpecs, String paramName) + { + this.shardSpecs = shardSpecs; + } + + @Test + public void testIsComplete() + { + final PartitionHolder holder = new PartitionHolder<>( + shardSpecs.get(0).createChunk(new OvershadowableInteger("version", shardSpecs.get(0).getPartitionNum(), 0)) + ); + for (int i = 0; i < shardSpecs.size() - 1; i++) { + Assert.assertFalse(holder.isComplete()); + final ShardSpec shardSpec = shardSpecs.get(i + 1); + holder.add(shardSpec.createChunk(new OvershadowableInteger("version", shardSpec.getPartitionNum(), 0))); + } + Assert.assertTrue(holder.isComplete()); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/RangeBucketShardSpecTest.java b/core/src/test/java/org/apache/druid/timeline/partition/RangeBucketShardSpecTest.java new file mode 100644 index 000000000000..d2c06e05f3f1 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/RangeBucketShardSpecTest.java @@ -0,0 +1,115 @@ +/* + * 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.timeline.partition; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.InjectableValues.Std; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.java.util.common.DateTimes; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class RangeBucketShardSpecTest +{ + @Test + public void testConvert() + { + Assert.assertEquals( + new BuildingSingleDimensionShardSpec(1, "dim", "start", "end", 5), + new RangeBucketShardSpec(1, "dim", "start", "end").convert(5) + ); + } + + @Test + public void testCreateChunk() + { + Assert.assertEquals( + new NumberedPartitionChunk<>(1, 0, "test"), + new RangeBucketShardSpec(1, "dim", "start", "end").createChunk("test") + ); + } + + @Test + public void testShardSpecLookup() + { + final List shardSpecs = ImmutableList.of( + new RangeBucketShardSpec(0, "dim", null, "c"), + new RangeBucketShardSpec(1, "dim", "f", "i"), + new RangeBucketShardSpec(2, "dim", "i", null) + ); + final ShardSpecLookup lookup = shardSpecs.get(0).getLookup(shardSpecs); + final long currentTime = DateTimes.nowUtc().getMillis(); + Assert.assertEquals( + shardSpecs.get(0), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "a", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(1), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "g", "time", currentTime) + ) + ) + ); + Assert.assertEquals( + shardSpecs.get(2), + lookup.getShardSpec( + currentTime, + new MapBasedInputRow( + currentTime, + ImmutableList.of("dim"), ImmutableMap.of("dim", "k", "time", currentTime) + ) + ) + ); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper mapper = ShardSpecTestUtils.initObjectMapper(); + mapper.registerSubtypes(new NamedType(RangeBucketShardSpec.class, RangeBucketShardSpec.TYPE)); + mapper.setInjectableValues(new Std().addValue(ObjectMapper.class, mapper)); + final RangeBucketShardSpec original = new RangeBucketShardSpec(1, "dim", "start", "end"); + final String json = mapper.writeValueAsString(original); + final RangeBucketShardSpec fromJson = (RangeBucketShardSpec) mapper.readValue(json, ShardSpec.class); + Assert.assertEquals(original, fromJson); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(RangeBucketShardSpec.class).usingGetClass().verify(); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/ShardSpecTestUtils.java b/core/src/test/java/org/apache/druid/timeline/partition/ShardSpecTestUtils.java new file mode 100644 index 000000000000..2f153651513b --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/partition/ShardSpecTestUtils.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.timeline.partition; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.MapperFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; + +public class ShardSpecTestUtils +{ + public static ObjectMapper initObjectMapper() + { + // Copied configurations from org.apache.druid.jackson.DefaultObjectMapper + final ObjectMapper mapper = new ObjectMapper(); + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + mapper.configure(MapperFeature.AUTO_DETECT_GETTERS, false); + // See https://github.com/FasterXML/jackson-databind/issues/170 + // configure(MapperFeature.AUTO_DETECT_CREATORS, false); + mapper.configure(MapperFeature.AUTO_DETECT_FIELDS, false); + mapper.configure(MapperFeature.AUTO_DETECT_IS_GETTERS, false); + mapper.configure(MapperFeature.AUTO_DETECT_SETTERS, false); + mapper.configure(MapperFeature.ALLOW_FINAL_FIELDS_AS_MUTATORS, false); + mapper.configure(SerializationFeature.INDENT_OUTPUT, false); + mapper.configure(SerializationFeature.FLUSH_AFTER_WRITE_VALUE, false); + return mapper; + } + + private ShardSpecTestUtils() + { + } +} diff --git a/dev/code-review/code-coverage.md b/dev/code-review/code-coverage.md new file mode 100644 index 000000000000..b5c52391452d --- /dev/null +++ b/dev/code-review/code-coverage.md @@ -0,0 +1,38 @@ + + +# Druid's Code Coverage Enforcement + +Druid code repository has an automated way of checking if new code has enough code coverage. +Druid CI checks are configured to enforce code coverage using JaCoCo. The CI checks will prevent a PR from being merged +if test coverage of new added code is below the set threshold. The CI checks filters test coverage based on a diff from +your PR and make sure that the thresholds are met. Druid currently enforce branch and line code coverage. + +However, do note that our current code coverage checks are merely smoke tests. They only verify that a line or branch +of code has been called during the test, but not that the functionality has been tested sufficiently. +Reviewers should still verify that all the different branches are sufficiently tested by reviewing the tests. + +## Running code coverage locally +Code coverage should be run locally to make sure the PR will pass Druid CI checks. +1. Code coverage on the codebase can be generated directly in [Intellij](../intellij-setup.md#Set-Code-Coverage-Runner). +2. Code coverage on just the diff of your PR can be generated in your terminal. First, you will have to install +diff-test-coverage by running `npm install @connectis/diff-test-coverage`. Next, run the unit tests +for the module you are working on `mvn -pl test jacoco:report` +(this will create a HTML report in target/site/jacoco/index.html). Finally, run +`git diff master...HEAD | diff-test-coverage --coverage "**/target/site/jacoco/jacoco.xml" --type jacoco --log-template "full" --` diff --git a/dev/intellij-images/code_coverage_1.png b/dev/intellij-images/code_coverage_1.png new file mode 100644 index 000000000000..439cb05d90ab Binary files /dev/null and b/dev/intellij-images/code_coverage_1.png differ diff --git a/dev/intellij-images/code_coverage_2.png b/dev/intellij-images/code_coverage_2.png new file mode 100644 index 000000000000..8eb0c8862949 Binary files /dev/null and b/dev/intellij-images/code_coverage_2.png differ diff --git a/dev/intellij-setup.md b/dev/intellij-setup.md index 9d455d1fd995..977f92fceb6c 100644 --- a/dev/intellij-setup.md +++ b/dev/intellij-setup.md @@ -34,6 +34,21 @@ an alias name. You can do this in Using `File` -> `Project Structure...` -> `Pla ## Code Style The Code Style is available in XML format at [druid_intellij_formatting.xml](druid_intellij_formatting.xml) and can be [imported into IntelliJ](https://www.jetbrains.com/help/idea/2017.1/copying-code-style-settings.html). +## Set Code Coverage Runner +Druid CI checks are configured to enforce code coverage using JaCoCo. The checks will prevent PR from being merged +if test coverage of new added code is below the set threshold. You should run the tests locally to make sure that +your code pass the coverage threshold. In IntelliJ, edit the template of JUnit tests in "Run Configurations..." +to use Jacoco for code coverage (this provides branch coverage instead of the default which is only line coverage). +You can then right click src/test/java folder of the modules you are modifying and click run with coverage. This +will generate a report to show the current code coverage on the code (not just your change). +![Code Coverage Runner Setup 1](intellij-images/code_coverage_1.png) +![Code Coverage Runner Setup 2](intellij-images/code_coverage_2.png) + +## Git Checkstyle Verification Hook (Optional) +Git Checkstyle pre-commit hook can be installed to automatically run checkstyle verification before committing, +saving cycle from avoiding the checkstyle failing later in Travis/CI environment. +The hook can be setup easily by running the /setup-hooks.sh script. + ## Metadata The installation of a MySQL metadata store is outside the scope of this document, but instructions on setting up MySQL can be found at [docs/development/extensions-core/mysql.md](/docs/development/extensions-core/mysql.md). This assumes you followed the example there and have a database named `druid` with proper permissions for a user named `druid` and a password of `diurd`. diff --git a/docs/assets/native-queries-01.png b/docs/assets/native-queries-01.png new file mode 100644 index 000000000000..27fd29b632cb Binary files /dev/null and b/docs/assets/native-queries-01.png differ diff --git a/docs/assets/tutorial-query-01.png b/docs/assets/tutorial-query-01.png index 8884d9bc8c47..371d35dec200 100644 Binary files a/docs/assets/tutorial-query-01.png and b/docs/assets/tutorial-query-01.png differ diff --git a/docs/assets/tutorial-query-02.png b/docs/assets/tutorial-query-02.png index cf407e3c631e..f459bd69961f 100644 Binary files a/docs/assets/tutorial-query-02.png and b/docs/assets/tutorial-query-02.png differ diff --git a/docs/assets/tutorial-query-03.png b/docs/assets/tutorial-query-03.png index c3b599c759c3..29443edb2634 100644 Binary files a/docs/assets/tutorial-query-03.png and b/docs/assets/tutorial-query-03.png differ diff --git a/docs/assets/tutorial-query-035.png b/docs/assets/tutorial-query-035.png new file mode 100644 index 000000000000..f22db18f19dc Binary files /dev/null and b/docs/assets/tutorial-query-035.png differ diff --git a/docs/assets/tutorial-query-04.png b/docs/assets/tutorial-query-04.png index 3f800a6c2ae9..9cfef34ccd01 100644 Binary files a/docs/assets/tutorial-query-04.png and b/docs/assets/tutorial-query-04.png differ diff --git a/docs/assets/tutorial-query-06.png b/docs/assets/tutorial-query-06.png index 60b4e1addacd..705605607a88 100644 Binary files a/docs/assets/tutorial-query-06.png and b/docs/assets/tutorial-query-06.png differ diff --git a/docs/assets/tutorial-query-07.png b/docs/assets/tutorial-query-07.png index d2e5a85b4f62..5997ba2f762d 100644 Binary files a/docs/assets/tutorial-query-07.png and b/docs/assets/tutorial-query-07.png differ diff --git a/docs/assets/tutorial-query-08.png b/docs/assets/tutorial-query-08.png new file mode 100644 index 000000000000..32aa5d48a7e4 Binary files /dev/null and b/docs/assets/tutorial-query-08.png differ diff --git a/docs/configuration/index.md b/docs/configuration/index.md index e1a9ddeac992..b17b74dbd80d 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -71,7 +71,7 @@ There are four JVM parameters that we set on all of our processes: 1. `-Duser.timezone=UTC` This sets the default timezone of the JVM to UTC. We always set this and do not test with other default timezones, so local timezones might work, but they also might uncover weird and interesting bugs. To issue queries in a non-UTC timezone, see [query granularities](../querying/granularities.html#period-granularities) 2. `-Dfile.encoding=UTF-8` This is similar to timezone, we test assuming UTF-8. Local encodings might work, but they also might result in weird and interesting bugs. -3. `-Djava.io.tmpdir=` Various parts of the system that interact with the file system do it via temporary files, and these files can get somewhat large. Many production systems are set up to have small (but fast) `/tmp` directories, which can be problematic with Druid so we recommend pointing the JVM’s tmp directory to something with a little more meat. +3. `-Djava.io.tmpdir=` Various parts of the system that interact with the file system do it via temporary files, and these files can get somewhat large. Many production systems are set up to have small (but fast) `/tmp` directories, which can be problematic with Druid so we recommend pointing the JVM’s tmp directory to something with a little more meat. This directory should not be volatile tmpfs. This directory should also have good read and write speed and hence NFS mount should strongly be avoided. 4. `-Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager` This allows log4j2 to handle logs for non-log4j2 components (like jetty) which use standard java logging. ### Extensions @@ -709,7 +709,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.manager.config.pollDuration`|How often the manager polls the config table for updates.|PT1M| |`druid.manager.segments.pollDuration`|The duration between polls the Coordinator does for updates to the set of active segments. Generally defines the amount of lag time it can take for the Coordinator to notice new segments.|PT1M| |`druid.manager.rules.pollDuration`|The duration between polls the Coordinator does for updates to the set of active rules. Generally defines the amount of lag time it can take for the Coordinator to notice rules.|PT1M| -|`druid.manager.rules.defaultTier`|The default tier from which default rules will be loaded from.|_default| +|`druid.manager.rules.defaultRule`|The default rule for the cluster|_default| |`druid.manager.rules.alertThreshold`|The duration after a failed poll upon which an alert should be emitted.|PT10M| #### Dynamic Configuration @@ -1204,7 +1204,7 @@ You can optionally configure caching to be enabled on the peons by setting cachi |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`| -|`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| +|`druid.realtime.cache.maxEntrySize`|positive integer|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1344,7 +1344,7 @@ You can optionally configure caching to be enabled on the Indexer by setting cac |`druid.realtime.cache.useCache`|true, false|Enable the cache on the realtime.|false| |`druid.realtime.cache.populateCache`|true, false|Populate the cache on the realtime.|false| |`druid.realtime.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`| -|`druid.realtime.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| +|`druid.realtime.cache.maxEntrySize`|positive integer|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. @@ -1383,7 +1383,7 @@ These Historical configurations can be defined in the `historical/runtime.proper |`druid.segmentCache.dropSegmentDelayMillis`|How long a process delays before completely dropping segment.|30000 (30 seconds)| |`druid.segmentCache.infoDir`|Historical processes keep track of the segments they are serving so that when the process is restarted they can reload the same segments without waiting for the Coordinator to reassign. This path defines where this metadata is kept. Directory will be created if needed.|${first_location}/info_dir| |`druid.segmentCache.announceIntervalMillis`|How frequently to announce segments while segments are loading from cache. Set this value to zero to wait for all segments to be loaded before announcing.|5000 (5 seconds)| -|`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from deep storage. Note that the work of loading segments involves downloading segments from deep storage, decompressing them and loading them to a memory mapped location. So the work is not all I/O Bound. Depending on CPU and network load, one could possibly increase this config to a higher value.|Number of cores| +|`druid.segmentCache.numLoadingThreads`|How many segments to drop or load concurrently from deep storage. Note that the work of loading segments involves downloading segments from deep storage, decompressing them and loading them to a memory mapped location. So the work is not all I/O Bound. Depending on CPU and network load, one could possibly increase this config to a higher value.|max(1,Number of cores / 6)| |`druid.segmentCache.numBootstrapThreads`|How many segments to load concurrently during historical startup.|`druid.segmentCache.numLoadingThreads`| |`druid.segmentCache.lazyLoadOnStart`|Whether or not to load segment columns metadata lazily during historical startup. When set to true, Historical startup time will be dramatically improved by deferring segment loading until the first time that segment takes part in a query, which will incur this cost instead. One catch is that if historical crashes while in the process of downloading and creating segment files, it is possible to end up with a corrupted segment on disk, this requires manual intervention to delete corrupted files. When the flag is set to true, historical startup would complete successfully and queries using this segment would fail at runtime.|false| |`druid.coordinator.loadqueuepeon.curator.numCallbackThreads`|Number of threads for executing callback actions associated with loading or dropping of segments. One might want to increase this number when noticing clusters are lagging behind w.r.t. balancing segments across historical nodes.|2| @@ -1650,7 +1650,7 @@ You can optionally only configure caching to be enabled on the Broker by setting |`druid.broker.cache.resultLevelCacheLimit`|positive integer|Maximum size of query response that can be cached.|`Integer.MAX_VALUE`| |`druid.broker.cache.unCacheable`|All druid query types|All query types to not cache.|`[]`| |`druid.broker.cache.cacheBulkMergeLimit`|positive integer or 0|Queries with more segments than this number will not attempt to fetch from cache at the broker level, leaving potential caching fetches (and cache result merging) to the Historicals|`Integer.MAX_VALUE`| -|`druid.broker.cache.maxEntrySize`|Maximum cache entry size in bytes.|1_000_000| +|`druid.broker.cache.maxEntrySize`|positive integer|Maximum cache entry size in bytes.|1_000_000| See [cache configuration](#cache-configuration) for how to configure cache settings. diff --git a/docs/design/processes.md b/docs/design/processes.md index bbcd917145dd..4c1e46a46a77 100644 --- a/docs/design/processes.md +++ b/docs/design/processes.md @@ -78,7 +78,7 @@ caller. End users typically query Brokers rather than querying Historicals or Mi Overlords, and Coordinators. They are optional since you can also simply contact the Druid Brokers, Overlords, and Coordinators directly. -The Router also runs the [Druid Console](../operations/management-uis.html#druid-console), a management UI for datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console. +The Router also runs the [Druid Console](../operations/druid-console.md), a management UI for datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console. ### Data server diff --git a/docs/design/router.md b/docs/design/router.md index c5f73084a873..cc037e229da8 100644 --- a/docs/design/router.md +++ b/docs/design/router.md @@ -24,13 +24,13 @@ title: "Router Process" > The Router is an optional and [experimental](../development/experimental.md) feature due to the fact that its recommended place in the Druid cluster architecture is still evolving. -> However, it has been battle-tested in production, and it hosts the powerful [Druid Console](../operations/management-uis.html#druid-console), so you should feel safe deploying it. +> However, it has been battle-tested in production, and it hosts the powerful [Druid Console](../operations/druid-console.md), so you should feel safe deploying it. The Apache Druid Router process can be used to route queries to different Broker processes. By default, the broker routes queries based on how [Rules](../operations/rule-configuration.md) are set up. For example, if 1 month of recent data is loaded into a `hot` cluster, queries that fall within the recent month can be routed to a dedicated set of brokers. Queries outside this range are routed to another set of brokers. This set up provides query isolation such that queries for more important data are not impacted by queries for less important data. For query routing purposes, you should only ever need the Router process if you have a Druid cluster well into the terabyte range. -In addition to query routing, the Router also runs the [Druid Console](../operations/management-uis.html#druid-console), a management UI for datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console. +In addition to query routing, the Router also runs the [Druid Console](../operations/druid-console.md), a management UI for datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console. ### Configuration diff --git a/docs/development/extensions-contrib/prometheus.md b/docs/development/extensions-contrib/prometheus.md new file mode 100644 index 000000000000..5f695f23c5cd --- /dev/null +++ b/docs/development/extensions-contrib/prometheus.md @@ -0,0 +1,70 @@ +--- +id: prometheus +title: "Prometheus Emitter" +--- + + + + +To use this Apache Druid (incubating) extension, make sure to [include](../../development/extensions.md#loading-extensions) `prometheus-emitter` extension. + +## Introduction + +This extension exposes Druid metrics for collection by a Prometheus server +(https://prometheus.io/) + +## Configuration + +All the configuration parameters for the Prometheus emitter are under `druid.emitter.prometheus`. + +|property|description|required?|default| +|--------|-----------|---------|-------| +|`druid.emitter.prometheus.strategy`|The strategy to expose prometheus metrics. Default strategy `exporter` would expose metrics for scraping purpose. Only peon task (short-lived jobs) need to use `pushgateway` strategy.|yes|exporter| +|`druid.emitter.prometheus.port`|The port on which to expose the prometheus HTTPServer. Required if using exporter strategy.|no|none| +|`druid.emitter.prometheus.namespace`|Optional metric namespace. Must match the regex `[a-zA-Z_:][a-zA-Z0-9_:]*`|no|"druid"| +|`druid.emitter.prometheus.dimensionMapPath`|JSON file defining the Prometheus metric type, desired dimensions, help text, and conversionFactor for every Druid metric.|no|Default mapping provided. See below.| +|`druid.emitter.prometheus.pushGatewayAddress`|Pushgateway address. Required if using pushgateway strategy|no|none| + + +### Metric names + +All metric names and labels are reformatted to match Prometheus standards. +- For names: all characters which are not alphanumeric, underscores, or colons (matching `[^a-zA-Z_:][^a-zA-Z0-9_:]*`) are replaced with `_` +- For labels: all characters which are not alphanumeric or underscores (matching `[^a-zA-Z0-9_][^a-zA-Z0-9_]*`) are replaced with `_` + +### Metric mapping + +Each metric to be collected by Prometheus must specify a type, one of `[timer, counter, guage]`. Prometheus Emitter expects this mapping to +be provided as a JSON file. Additionally, this mapping specifies which dimensions should be included for each metric. Prometheus expects +histogram timers to use Seconds as the base unit. Timers which do not use seconds as a base unit can use the `conversionFactor` to set +the base time unit. If the user does not specify their own JSON file, a default mapping is used. All +metrics are expected to be mapped. Metrics which are not mapped will not be tracked. +Prometheus metric path is organized using the following schema: +` : { "dimensions" : , "type" : , conversionFactor: , "help" : ,}` +e.g. +`query/time" : { "dimensions" : ["dataSource", "type"], "conversionFactor": 1000.0, "type" : "timer", "help": "Seconds taken to complete a query."}` + +For metrics which are emitted from multiple services with different dimensions, the metric name is prefixed with +the service name. +e.g. +`"coordinator-segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge" }, + "historical-segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge" }` + +For most use-cases, the default mapping is sufficient. diff --git a/docs/ingestion/faq.md b/docs/ingestion/faq.md index 1e6ffe10254f..308407fa3426 100644 --- a/docs/ingestion/faq.md +++ b/docs/ingestion/faq.md @@ -66,6 +66,20 @@ Other common reasons that hand-off fails are as follows: Make sure to include the `druid-hdfs-storage` and all the hadoop configuration, dependencies (that can be obtained by running command `hadoop classpath` on a machine where hadoop has been setup) in the classpath. And, provide necessary HDFS settings as described in [deep storage](../dependencies/deep-storage.md) . +## How do I know when I can make query to Druid after submitting batch ingestion task? + +You can verify if segments created by a recent ingestion task are loaded onto historicals and available for querying using the following workflow. +1. Submit your ingestion task. +2. Repeatedly poll the [Overlord's tasks API](../operations/api-reference.md#tasks) ( `/druid/indexer/v1/task/{taskId}/status`) until your task is shown to be successfully completed. +3. Poll the [Segment Loading by Datasource API](../operations/api-reference.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with +`forceMetadataRefresh=true` and `interval=` once. +(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms of the load on the metadata store but is necessary to make sure that we verify all the latest segments' load status) +If there are segments not yet loaded, continue to step 4, otherwise you can now query the data. +4. Repeatedly poll the [Segment Loading by Datasource API](../operations/api-reference.md#segment-loading-by-datasource) (`/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus`) with +`forceMetadataRefresh=false` and `interval=`. +Continue polling until all segments are loaded. Once all segments are loaded you can now query the data. +Note that this workflow only guarantees that the segments are available at the time of the [Segment Loading by Datasource API](../operations/api-reference.md#segment-loading-by-datasource) call. Segments can still become missing because of historical process failures or any other reasons afterward. + ## I don't see my Druid segments on my Historical processes You can check the Coordinator console located at `:`. Make sure that your segments have actually loaded on [Historical processes](../design/historical.md). If your segments are not present, check the Coordinator logs for messages about capacity of replication errors. One reason that segments are not downloaded is because Historical processes have maxSizes that are too small, making them incapable of downloading more data. You can change that with (for example): diff --git a/docs/ingestion/hadoop.md b/docs/ingestion/hadoop.md index 2101c1b9fc9a..0f5db6a33b78 100644 --- a/docs/ingestion/hadoop.md +++ b/docs/ingestion/hadoop.md @@ -329,12 +329,12 @@ The tuningConfig is optional and default parameters will be used if no tuningCon |useCombiner|Boolean|Use Hadoop combiner to merge rows at mapper if possible.|no (default == false)| |jobProperties|Object|A map of properties to add to the Hadoop job configuration, see below for details.|no (default == null)| |indexSpec|Object|Tune how data is indexed. See [`indexSpec`](index.md#indexspec) on the main ingestion page for more information.|no| -|indexSpecForIntermediatePersists|defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. this can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. however, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [`indexSpec`](index.md#indexspec) for possible values.|no (default = same as indexSpec)| +|indexSpecForIntermediatePersists|Object|defines segment storage format options to be used at indexing time for intermediate persisted temporary segments. this can be used to disable dimension/metric compression on intermediate segments to reduce memory required for final merging. however, disabling compression on intermediate segments might increase page cache use while they are used before getting merged into final segment published, see [`indexSpec`](index.md#indexspec) for possible values.|no (default = same as indexSpec)| |numBackgroundPersistThreads|Integer|The number of new background threads to use for incremental persists. Using this feature causes a notable increase in memory pressure and CPU usage but will make the job finish more quickly. If changing from the default of 0 (use current thread for persists), we recommend setting it to 1.|no (default == 0)| |forceExtendableShardSpecs|Boolean|Forces use of extendable shardSpecs. Hash-based partitioning always uses an extendable shardSpec. For single-dimension partitioning, this option should be set to true to use an extendable shardSpec. For partitioning, please check [Partitioning specification](#partitionsspec). This option can be useful when you need to append more data to existing dataSource.|no (default = false)| |useExplicitVersion|Boolean|Forces HadoopIndexTask to use version.|no (default = false)| -|logParseExceptions|Boolean|If true, log an error message when a parsing exception occurs, containing information about the row where the error occurred.|false|no| -|maxParseExceptions|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overrides `ignoreInvalidRows` if `maxParseExceptions` is defined.|unlimited|no| +|logParseExceptions|Boolean|If true, log an error message when a parsing exception occurs, containing information about the row where the error occurred.|no(default = false)| +|maxParseExceptions|Integer|The maximum number of parse exceptions that can occur before the task halts ingestion and fails. Overrides `ignoreInvalidRows` if `maxParseExceptions` is defined.|no(default = unlimited)| |useYarnRMJobStatusFallback|Boolean|If the Hadoop jobs created by the indexing task are unable to retrieve their completion status from the JobHistory server, and this parameter is true, the indexing task will try to fetch the application status from `http:///ws/v1/cluster/apps/`, where `` is the value of `yarn.resourcemanager.webapp.address` in your Hadoop configuration. This flag is intended as a fallback for cases where an indexing task's jobs succeed, but the JobHistory server is unavailable, causing the indexing task to fail because it cannot determine the job statuses.|no (default = true)| ### `jobProperties` diff --git a/docs/ingestion/native-batch.md b/docs/ingestion/native-batch.md index 02c8a321f984..799b32e0fbad 100644 --- a/docs/ingestion/native-batch.md +++ b/docs/ingestion/native-batch.md @@ -53,7 +53,7 @@ The detailed behavior of the Parallel task is different depending on the [`parti See each `partitionsSpec` for more details. To use this task, the [`inputSource`](#input-sources) in the `ioConfig` should be _splittable_ and `maxNumConcurrentSubTasks` should be set to larger than 1 in the `tuningConfig`. -Otherwise, this task runs sequentially; the `index_paralllel` task reads each input file one by one and creates segments by itself. +Otherwise, this task runs sequentially; the `index_parallel` task reads each input file one by one and creates segments by itself. The supported splittable input formats for now are: - [`s3`](#s3-input-source) reads data from AWS S3 storage. @@ -63,6 +63,7 @@ The supported splittable input formats for now are: - [`http`](#http-input-source) reads data from HTTP servers. - [`local`](#local-input-source) reads data from local storage. - [`druid`](#druid-input-source) reads data from a Druid datasource. +- [`sql`](#sql-input-source) reads data from a RDBMS source. Some other cloud storage types are supported with the legacy [`firehose`](#firehoses-deprecated). The below `firehose` types are also splittable. Note that only text formats are supported @@ -301,7 +302,7 @@ and then by the hash value of `partitionDimensions` (secondary partition key) in The partitioned data is stored in local storage of the [middleManager](../design/middlemanager.md) or the [indexer](../design/indexer.md). - The `partial segment merge` phase is similar to the Reduce phase in MapReduce. -The Parallel task spawns a new set of worker tasks (type `partial_index_merge`) to merge the partitioned data +The Parallel task spawns a new set of worker tasks (type `partial_index_generic_merge`) to merge the partitioned data created in the previous phase. Here, the partitioned data is shuffled based on the time chunk and the hash value of `partitionDimensions` to be merged; each worker task reads the data falling in the same time chunk and the same hash value from multiple MiddleManager/Indexer processes and merges @@ -1310,6 +1311,59 @@ A spec that applies a filter and reads a subset of the original datasource's col This spec above will only return the `page`, `user` dimensions and `added` metric. Only rows where `page` = `Druid` will be returned. +### SQL Input Source + +The SQL input source is used to read data directly from RDBMS. +The SQL input source is _splittable_ and can be used by the [Parallel task](#parallel-task), where each worker task will read from one SQL query from the list of queries. +Since this input source has a fixed input format for reading events, no `inputFormat` field needs to be specified in the ingestion spec when using this input source. +Please refer to the Recommended practices section below before using this input source. + +|property|description|required?| +|--------|-----------|---------| +|type|This should be "sql".|Yes| +|database|Specifies the database connection details. The database type corresponds to the extension that supplies the `connectorConfig` support and this extension must be loaded into Druid. For database types `mysql` and `postgresql`, the `connectorConfig` support is provided by [mysql-metadata-storage](../development/extensions-core/mysql.md) and [postgresql-metadata-storage](../development/extensions-core/postgresql.md) extensions respectively.|Yes| +|foldCase|Toggle case folding of database column names. This may be enabled in cases where the database returns case insensitive column names in query results.|No| +|sqls|List of SQL queries where each SQL query would retrieve the data to be indexed.|Yes| + +An example SqlInputSource spec is shown below: + +```json +... + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "sql", + "database": { + "type": "mysql", + "connectorConfig": { + "connectURI": "jdbc:mysql://host:port/schema", + "user": "user", + "password": "password" + } + }, + "sqls": ["SELECT * FROM table1 WHERE timestamp BETWEEN '2013-01-01 00:00:00' AND '2013-01-01 11:59:59'", "SELECT * FROM table2 WHERE timestamp BETWEEN '2013-01-01 00:00:00' AND '2013-01-01 11:59:59'"] + }, +... +``` + +The spec above will read all events from two separate SQLs for the interval `2013-01-01/2013-01-02`. +Each of the SQL queries will be run in its own sub-task and thus for the above example, there would be two sub-tasks. + +**Recommended practices** + +Compared to the other native batch InputSources, SQL InputSource behaves differently in terms of reading the input data and so it would be helpful to consider the following points before using this InputSource in a production environment: + +* During indexing, each sub-task would execute one of the SQL queries and the results are stored locally on disk. The sub-tasks then proceed to read the data from these local input files and generate segments. Presently, there isn’t any restriction on the size of the generated files and this would require the MiddleManagers or Indexers to have sufficient disk capacity based on the volume of data being indexed. + +* Filtering the SQL queries based on the intervals specified in the `granularitySpec` can avoid unwanted data being retrieved and stored locally by the indexing sub-tasks. For example, if the `intervals` specified in the `granularitySpec` is `["2013-01-01/2013-01-02"]` and the SQL query is `SELECT * FROM table1`, `SqlInputSource` will read all the data for `table1` based on the query, even though only data between the intervals specified will be indexed into Druid. + +* Pagination may be used on the SQL queries to ensure that each query pulls a similar amount of data, thereby improving the efficiency of the sub-tasks. + +* Similar to file-based input formats, any updates to existing data will replace the data in segments specific to the intervals specified in the `granularitySpec`. + + +### + ## Firehoses (Deprecated) Firehoses are deprecated in 0.17.0. It's highly recommended to use the [Input source](#input-sources) instead. @@ -1544,6 +1598,7 @@ This firehose will accept any type of parser, but will only utilize the list of This Firehose can be used to ingest events residing in an RDBMS. The database connection information is provided as part of the ingestion spec. For each query, the results are fetched locally and indexed. If there are multiple queries from which data needs to be indexed, queries are prefetched in the background, up to `maxFetchCapacityBytes` bytes. +This Firehose is _splittable_ and can be used by [native parallel index tasks](native-batch.md#parallel-task). This firehose will accept any type of parser, but will only utilize the list of dimensions and the timestamp specification. See the extension documentation for more detailed ingestion examples. Requires one of the following extensions: diff --git a/docs/misc/math-expr.md b/docs/misc/math-expr.md index e4ad694738f8..dc356479ad58 100644 --- a/docs/misc/math-expr.md +++ b/docs/misc/math-expr.md @@ -76,7 +76,8 @@ The following built-in functions are available. |like|like(expr, pattern[, escape]) is equivalent to SQL `expr LIKE pattern`| |lookup|lookup(expr, lookup-name) looks up expr in a registered [query-time lookup](../querying/lookups.md)| |parse_long|parse_long(string[, radix]) parses a string as a long with the given radix, or 10 (decimal) if a radix is not provided.| -|regexp_extract|regexp_extract(expr, pattern[, index]) applies a regular expression pattern and extracts a capture group index, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern.| +|regexp_extract|regexp_extract(expr, pattern[, index]) applies a regular expression pattern and extracts a capture group index, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern.| +|regexp_like|regexp_like(expr, pattern) returns whether `expr` matches regular expression `pattern`. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. | |replace|replace(expr, pattern, replacement) replaces pattern with replacement| |substring|substring(expr, index, length) behaves like java.lang.String's substring| |right|right(expr, length) returns the rightmost length characters from a string| @@ -90,8 +91,8 @@ The following built-in functions are available. |upper|upper(expr) converts a string to uppercase| |reverse|reverse(expr) reverses a string| |repeat|repeat(expr, N) repeats a string N times| -|lpad|lpad(expr, length, chars) returns a string of `length` from `expr` left-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. If either `expr` or `chars` are null, the result will be null.| -|rpad|rpad(expr, length, chars) returns a string of `length` from `expr` right-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. If either `expr` or `chars` are null, the result will be null.| +|lpad|lpad(expr, length, chars) returns a string of `length` from `expr` left-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. The result will be null if either `expr` or `chars` is null. If `chars` is an empty string, no padding is added, however `expr` may be trimmed if necessary.| +|rpad|rpad(expr, length, chars) returns a string of `length` from `expr` right-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. The result will be null if either `expr` or `chars` is null. If `chars` is an empty string, no padding is added, however `expr` may be trimmed if necessary.| ## Time functions @@ -140,7 +141,7 @@ See javadoc of java.lang.Math for detailed explanation for each function. |pow|pow(x, y) would return the value of the x raised to the power of y| |remainder|remainder(x, y) would return the remainder operation on two arguments as prescribed by the IEEE 754 standard| |rint|rint(x) would return value that is closest in value to x and is equal to a mathematical integer| -|round|round(x, y) would return the value of the x rounded to the y decimal places. While x can be an integer or floating-point number, y must be an integer. The type of the return value is specified by that of x. y defaults to 0 if omitted. When y is negative, x is rounded on the left side of the y decimal points.| +|round|round(x, y) would return the value of the x rounded to the y decimal places. While x can be an integer or floating-point number, y must be an integer. The type of the return value is specified by that of x. y defaults to 0 if omitted. When y is negative, x is rounded on the left side of the y decimal points. If x is `NaN`, x will return 0. If x is infinity, x will be converted to the nearest finite double. | |scalb|scalb(d, sf) would return d * 2^sf rounded as if performed by a single correctly rounded floating-point multiply to a member of the double value set| |signum|signum(x) would return the signum function of the argument x| |sin|sin(x) would return the trigonometric sine of an angle x| diff --git a/docs/operations/api-reference.md b/docs/operations/api-reference.md index a66dd649f410..a3610a8bc52a 100644 --- a/docs/operations/api-reference.md +++ b/docs/operations/api-reference.md @@ -96,11 +96,11 @@ Returns the percentage of segments actually loaded in the cluster versus segment * `/druid/coordinator/v1/loadstatus?simple` -Returns the number of segments left to load until segments that should be loaded in the cluster are available for queries. This does not include replication. +Returns the number of segments left to load until segments that should be loaded in the cluster are available for queries. This does not include segment replication counts. * `/druid/coordinator/v1/loadstatus?full` -Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes replication. +Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available. This includes segment replication counts. * `/druid/coordinator/v1/loadqueue` @@ -114,6 +114,45 @@ Returns the number of segments to load and drop, as well as the total segment lo Returns the serialized JSON of segments to load and drop for each Historical process. + +#### Segment Loading by Datasource + +Note that all _interval_ query parameters are ISO 8601 strings (e.g., 2016-06-27/2016-06-28). +Also note that these APIs only guarantees that the segments are available at the time of the call. +Segments can still become missing because of historical process failures or any other reasons afterward. + +##### GET + +* `/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?forceMetadataRefresh={boolean}&interval={myInterval}` + +Returns the percentage of segments actually loaded in the cluster versus segments that should be loaded in the cluster for the given +datasource over the given interval (or last 2 weeks if interval is not given). `forceMetadataRefresh` is required to be set. +Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store +(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms +of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) +Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. +If no used segments are found for the given inputs, this API returns `204 No Content` + + * `/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?simple&forceMetadataRefresh={boolean}&interval={myInterval}` + +Returns the number of segments left to load until segments that should be loaded in the cluster are available for the given datasource +over the given interval (or last 2 weeks if interval is not given). This does not include segment replication counts. `forceMetadataRefresh` is required to be set. +Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store +(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms +of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) +Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. +If no used segments are found for the given inputs, this API returns `204 No Content` + +* `/druid/coordinator/v1/datasources/{dataSourceName}/loadstatus?full&forceMetadataRefresh={boolean}&interval={myInterval}` + +Returns the number of segments left to load in each tier until segments that should be loaded in the cluster are all available for the given datasource +over the given interval (or last 2 weeks if interval is not given). This includes segment replication counts. `forceMetadataRefresh` is required to be set. +Setting `forceMetadataRefresh` to true will force the coordinator to poll latest segment metadata from the metadata store +(Note: `forceMetadataRefresh=true` refreshes Coordinator's metadata cache of all datasources. This can be a heavy operation in terms +of the load on the metadata store but can be necessary to make sure that we verify all the latest segments' load status) +Setting `forceMetadataRefresh` to false will use the metadata cached on the coordinator from the last force/periodic refresh. +If no used segments are found for the given inputs, this API returns `204 No Content` + #### Metadata store information ##### GET diff --git a/docs/operations/basic-cluster-tuning.md b/docs/operations/basic-cluster-tuning.md index 7b62e2bf0128..6fee5ffe12a9 100644 --- a/docs/operations/basic-cluster-tuning.md +++ b/docs/operations/basic-cluster-tuning.md @@ -389,7 +389,7 @@ Enabling process termination on out-of-memory errors is useful as well, since th ``` -Duser.timezone=UTC -Dfile.encoding=UTF-8 --Djava.io.tmpdir= +-Djava.io.tmpdir= -Djava.util.logging.manager=org.apache.logging.log4j.jul.LogManager -Dorg.jboss.logging.provider=slf4j -Dnet.spy.log.LoggerImpl=net.spy.memcached.compat.log.SLF4JLogger diff --git a/docs/operations/druid-console.md b/docs/operations/druid-console.md index 24856e7afdd6..2bda0e92250a 100644 --- a/docs/operations/druid-console.md +++ b/docs/operations/druid-console.md @@ -22,20 +22,25 @@ title: "Web console" ~ under the License. --> +Druid include a console for managing datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. Users can also run SQL and native Druid queries in the console. The Druid Console is hosted by the [Router](../design/router.md) process. -In addition, the following cluster settings must be enabled: +The following cluster settings must be enabled, as they are by default: - the Router's [management proxy](../design/router.html#enabling-the-management-proxy) must be enabled. - the Broker processes in the cluster must have [Druid SQL](../querying/sql.md) enabled. -After enabling Druid SQL on the Brokers and deploying a Router with the management proxy enabled, the Druid console can be accessed at: +The Druid console can be accessed at: ``` http://: ``` +> It is important to note that any Druid console user will have, effectively, the same file permissions as the user under which Druid runs. One way these permissions are surfaced is in the file browser dialog. The dialog +will show console users the files that the underlying user has permissions to. In general, avoid running Druid as +root user. Consider creating a dedicated user account for running Druid. + Below is a description of the high-level features and functionality of the Druid Console ## Home diff --git a/docs/operations/management-uis.md b/docs/operations/management-uis.md index bb3e80bf5e88..2b1d23470351 100644 --- a/docs/operations/management-uis.md +++ b/docs/operations/management-uis.md @@ -1,6 +1,6 @@ --- id: management-uis -title: "Management UIs" +title: "Legacy Management UIs" --- -## Druid console +## Legacy consoles Druid provides a console for managing datasources, segments, tasks, data processes (Historicals and MiddleManagers), and coordinator dynamic configuration. The user can also run SQL and native Druid queries within the console. -The Druid Console is hosted by the [Router](../design/router.md) process. We recommend running the Router process on your [Query server](../design/processes.md). - -In addition, the following cluster settings must be enabled: - -- the Router's [management proxy](../design/router.html#enabling-the-management-proxy) must be enabled. -- the Broker processes in the cluster must have [Druid SQL](../querying/sql.md) enabled. - -After enabling Druid SQL on the Brokers and deploying a Router with the management proxy enabled, the Druid console can be accessed at: - -``` -http://: -``` +For more information on the Druid Console, have a look at the [Druid Console overview](./druid-console.md) The Druid Console contains all of the functionality provided by the older consoles described below, which are still available if needed. The legacy consoles may be replaced by the Druid Console in the future. -For more information on the features of the Druid Console have a look at the [Druid Console overview](./druid-console.md) - -## Legacy consoles - These older consoles provide a subset of the functionality of the Druid Console. We recommend using the Druid Console if possible. ### Coordinator consoles diff --git a/docs/operations/password-provider.md b/docs/operations/password-provider.md index 4a28e64aae8b..209b2d3882ea 100644 --- a/docs/operations/password-provider.md +++ b/docs/operations/password-provider.md @@ -33,7 +33,7 @@ Environment variable password provider provides password by looking at specified e.g ```json -{ "type": "environment", "variable": "METADATA_STORAGE_PASSWORD" } +druid.metadata.storage.connector.password={ "type": "environment", "variable": "METADATA_STORAGE_PASSWORD" } ``` The values are described below. @@ -50,5 +50,5 @@ Please have a look at "Adding a new Password Provider implementation" on this [p To use this implementation, simply set the relevant password runtime property to something similar as was done for Environment variable password provider like - ```json -{ "type": "", "": "", ... } +druid.metadata.storage.connector.password={ "type": "", "": "", ... } ``` diff --git a/docs/operations/rule-configuration.md b/docs/operations/rule-configuration.md index 4228994a30d3..1ea074f72394 100644 --- a/docs/operations/rule-configuration.md +++ b/docs/operations/rule-configuration.md @@ -170,8 +170,9 @@ The interval of a segment will be compared against the specified period. The per ## Broadcast Rules -Broadcast rules indicate how segments of different datasources should be co-located in Historical processes. -Once a broadcast rule is configured for a datasource, all segments of the datasource are broadcasted to the servers holding _any segments_ of the co-located datasources. +Broadcast rules indicate that segments of a data source should be loaded by all servers of a cluster of the following types: historicals, brokers, tasks, and indexers. + +Note that the broadcast segments are only directly queryable through the historicals, but they are currently loaded on other server types to support join queries. ### Forever Broadcast Rule @@ -179,13 +180,13 @@ Forever broadcast rules are of the form: ```json { - "type" : "broadcastForever", - "colocatedDataSources" : [ "target_source1", "target_source2" ] + "type" : "broadcastForever" } ``` * `type` - this should always be "broadcastForever" -* `colocatedDataSources` - A JSON List containing datasource names to be co-located. `null` and empty list means broadcasting to every process in the cluster. + +This rule applies to all segments of a datasource, covering all intervals. ### Interval Broadcast Rule @@ -194,13 +195,11 @@ Interval broadcast rules are of the form: ```json { "type" : "broadcastByInterval", - "colocatedDataSources" : [ "target_source1", "target_source2" ], "interval" : "2012-01-01/2013-01-01" } ``` * `type` - this should always be "broadcastByInterval" -* `colocatedDataSources` - A JSON List containing datasource names to be co-located. `null` and empty list means broadcasting to every process in the cluster. * `interval` - A JSON Object representing ISO-8601 Periods. Only the segments of the interval will be broadcasted. ### Period Broadcast Rule @@ -210,22 +209,17 @@ Period broadcast rules are of the form: ```json { "type" : "broadcastByPeriod", - "colocatedDataSources" : [ "target_source1", "target_source2" ], "period" : "P1M", "includeFuture" : true } ``` * `type` - this should always be "broadcastByPeriod" -* `colocatedDataSources` - A JSON List containing datasource names to be co-located. `null` and empty list means broadcasting to every process in the cluster. * `period` - A JSON Object representing ISO-8601 Periods * `includeFuture` - A JSON Boolean indicating whether the load period should include the future. This property is optional, Default is true. The interval of a segment will be compared against the specified period. The period is from some time in the past to the future or to the current time, which depends on `includeFuture` is true or false. The rule matches if the period *overlaps* the interval. -> broadcast rules don't guarantee that segments of the datasources are always co-located because segments for the colocated datasources are not loaded together atomically. -> If you want to always co-locate the segments of some datasources together, it is recommended to leave colocatedDataSources empty. - ## Permanently deleting data Druid can fully drop data from the cluster, wipe the metadata store entry, and remove the data from deep storage for any diff --git a/docs/querying/datasource.md b/docs/querying/datasource.md index 66cc9058841a..d6c478d110e4 100644 --- a/docs/querying/datasource.md +++ b/docs/querying/datasource.md @@ -348,3 +348,5 @@ future versions: always be correct. - Performance-related optimizations as mentioned in the [previous section](#join-performance). - Join algorithms other than broadcast hash-joins. +- Join condition on a column compared to a constant value. +- Join conditions on a column containing a multi-value dimension. diff --git a/docs/querying/querying.md b/docs/querying/querying.md index 5a3e99a31755..82d91a763f12 100644 --- a/docs/querying/querying.md +++ b/docs/querying/querying.md @@ -35,6 +35,13 @@ posted like this: curl -X POST ':/druid/v2/?pretty' -H 'Content-Type:application/json' -H 'Accept:application/json' -d @ ``` +> Replace `:` with the appropriate address and port for your system. For example, if running the quickstart configuration, replace `:` with localhost:8888. + +You can also enter them directly in the Druid console's Query view. Simply pasting a native query into the console switches the editor into JSON mode. + +![Native query](../assets/native-queries-01.png "Native query") + + Druid's native query language is JSON over HTTP, although many members of the community have contributed different [client libraries](/libraries.html) in other languages to query Druid. @@ -44,7 +51,7 @@ The Content-Type/Accept Headers can also take 'application/x-jackson-smile'. curl -X POST ':/druid/v2/?pretty' -H 'Content-Type:application/json' -H 'Accept:application/x-jackson-smile' -d @ ``` -Note: If Accept header is not provided, it defaults to value of 'Content-Type' header. +> If the Accept header is not provided, it defaults to the value of 'Content-Type' header. Druid's native query is relatively low level, mapping closely to how computations are performed internally. Druid queries are designed to be lightweight and complete very quickly. This means that for more complex analysis, or to build diff --git a/docs/querying/sql.md b/docs/querying/sql.md index 80f8a399bf33..acbc3ae0e726 100644 --- a/docs/querying/sql.md +++ b/docs/querying/sql.md @@ -287,7 +287,7 @@ to FLOAT. At runtime, Druid will widen 32-bit floats to 64-bit for most expressi |`SQRT(expr)`|Square root.| |`TRUNCATE(expr[, digits])`|Truncate expr to a specific number of decimal digits. If digits is negative, then this truncates that many places to the left of the decimal point. Digits defaults to zero if not specified.| |`TRUNC(expr[, digits])`|Synonym for `TRUNCATE`.| -|`ROUND(expr[, digits])`|`ROUND(x, y)` would return the value of the x rounded to the y decimal places. While x can be an integer or floating-point number, y must be an integer. The type of the return value is specified by that of x. y defaults to 0 if omitted. When y is negative, x is rounded on the left side of the y decimal points.| +|`ROUND(expr[, digits])`|`ROUND(x, y)` would return the value of the x rounded to the y decimal places. While x can be an integer or floating-point number, y must be an integer. The type of the return value is specified by that of x. y defaults to 0 if omitted. When y is negative, x is rounded on the left side of the y decimal points. If `expr` evaluates to either `NaN`, `expr` will be converted to 0. If `expr` is infinity, `expr` will be converted to the nearest finite double. | |`x + y`|Addition.| |`x - y`|Subtraction.| |`x * y`|Multiplication.| @@ -322,7 +322,8 @@ String functions accept strings, and return a type appropriate to the function. |`LOWER(expr)`|Returns expr in all lowercase.| |`PARSE_LONG(string[, radix])`|Parses a string into a long (BIGINT) with the given radix, or 10 (decimal) if a radix is not provided.| |`POSITION(needle IN haystack [FROM fromIndex])`|Returns the index of needle within haystack, with indexes starting from 1. The search will begin at fromIndex, or 1 if fromIndex is not specified. If the needle is not found, returns 0.| -|`REGEXP_EXTRACT(expr, pattern, [index])`|Apply regular expression pattern and extract a capture group, or null if there is no match. If index is unspecified or zero, returns the substring that matched the pattern.| +|`REGEXP_EXTRACT(expr, pattern, [index])`|Apply regular expression `pattern` to `expr` and extract a capture group, or `NULL` if there is no match. If index is unspecified or zero, returns the first substring that matched the pattern. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. Note: when `druid.generic.useDefaultValueForNull = true`, it is not possible to differentiate an empty-string match from a non-match (both will return `NULL`).| +|`REGEXP_LIKE(expr, pattern)`|Returns whether `expr` matches regular expression `pattern`. The pattern may match anywhere inside `expr`; if you want to match the entire string instead, use the `^` and `$` markers at the start and end of your pattern. Similar to [`LIKE`](#comparison-operators), but uses regexps instead of LIKE patterns. Especially useful in WHERE clauses.| |`REPLACE(expr, pattern, replacement)`|Replaces pattern with replacement in expr, and returns the result.| |`STRPOS(haystack, needle)`|Returns the index of needle within haystack, with indexes starting from 1. If the needle is not found, returns 0.| |`SUBSTRING(expr, index, [length])`|Returns a substring of expr starting at index, with a max length, both measured in UTF-16 code units.| @@ -330,14 +331,14 @@ String functions accept strings, and return a type appropriate to the function. |`LEFT(expr, [length])`|Returns the leftmost length characters from expr.| |`SUBSTR(expr, index, [length])`|Synonym for SUBSTRING.| |TRIM([BOTH | LEADING | TRAILING] [ FROM] expr)|Returns expr with characters removed from the leading, trailing, or both ends of "expr" if they are in "chars". If "chars" is not provided, it defaults to " " (a space). If the directional argument is not provided, it defaults to "BOTH".| -|`BTRIM(expr[, chars])`|Alternate form of `TRIM(BOTH FROM `).| -|`LTRIM(expr[, chars])`|Alternate form of `TRIM(LEADING FROM `).| -|`RTRIM(expr[, chars])`|Alternate form of `TRIM(TRAILING FROM `).| +|`BTRIM(expr[, chars])`|Alternate form of `TRIM(BOTH FROM )`.| +|`LTRIM(expr[, chars])`|Alternate form of `TRIM(LEADING FROM )`.| +|`RTRIM(expr[, chars])`|Alternate form of `TRIM(TRAILING FROM )`.| |`UPPER(expr)`|Returns expr in all uppercase.| |`REVERSE(expr)`|Reverses expr.| |`REPEAT(expr, [N])`|Repeats expr N times| -|`LPAD(expr, length[, chars])`|Returns a string of "length" from "expr" left-padded with "chars". If "length" is shorter than the length of "expr", the result is "expr" which is truncated to "length". If either "expr" or "chars" are null, the result will be null.| -|`RPAD(expr, length[, chars])`|Returns a string of "length" from "expr" right-padded with "chars". If "length" is shorter than the length of "expr", the result is "expr" which is truncated to "length". If either "expr" or "chars" are null, the result will be null.| +|`LPAD(expr, length[, chars])`|Returns a string of `length` from `expr` left-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. The result will be null if either `expr` or `chars` is null. If `chars` is an empty string, no padding is added, however `expr` may be trimmed if necessary.| +|`RPAD(expr, length[, chars])`|Returns a string of `length` from `expr` right-padded with `chars`. If `length` is shorter than the length of `expr`, the result is `expr` which is truncated to `length`. The result will be null if either `expr` or `chars` is null. If `chars` is an empty string, no padding is added, however `expr` may be trimmed if necessary.| ### Time functions @@ -709,6 +710,8 @@ Druid does not support all SQL features. In particular, the following features a - JOIN between native datasources (table, lookup, subquery) and system tables. - JOIN conditions that are not an equality between expressions from the left- and right-hand sides. +- JOIN conditions containing a constant value inside the condition. +- JOIN conditions on a column which contains a multi-value dimension. - OVER clauses, and analytic functions such as `LAG` and `LEAD`. - OFFSET clauses. - DDL and DML. @@ -933,7 +936,7 @@ SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = 'druid' AND TABLE_ |Column|Notes| |------|-----| -|TABLE_CATALOG|Unused| +|TABLE_CATALOG|Always set as 'druid'| |TABLE_SCHEMA|| |TABLE_NAME|| |TABLE_TYPE|"TABLE" or "SYSTEM_TABLE"| @@ -942,7 +945,7 @@ SELECT * FROM INFORMATION_SCHEMA.COLUMNS WHERE TABLE_SCHEMA = 'druid' AND TABLE_ |Column|Notes| |------|-----| -|TABLE_CATALOG|Unused| +|TABLE_CATALOG|Always set as 'druid'| |TABLE_SCHEMA|| |TABLE_NAME|| |COLUMN_NAME|| diff --git a/docs/tutorials/index.md b/docs/tutorials/index.md index 2ee1b42eeb6e..5c8bdd422cab 100644 --- a/docs/tutorials/index.md +++ b/docs/tutorials/index.md @@ -54,6 +54,10 @@ The software requirements for the installation machine are: `DRUID_JAVA_HOME` if there is more than one instance of Java. To verify Java requirements for your environment, run the `bin/verify-java` script. +Before installing a production Druid instance, be sure to consider the user account on the operating system under +which Druid will run. This is important because any Druid console user will have, effectively, the same permissions as +that user. So, for example, the file browser UI will show console users the files that the underlying user can +access. In general, avoid running Druid as root user. Consider creating a dedicated user account for running Druid. ## Step 1. Install Druid diff --git a/docs/tutorials/tutorial-query.md b/docs/tutorials/tutorial-query.md index 4b19c92b89b3..19461ac82191 100644 --- a/docs/tutorials/tutorial-query.md +++ b/docs/tutorials/tutorial-query.md @@ -24,56 +24,164 @@ sidebar_label: "Querying data" --> -This tutorial will demonstrate how to query data in Apache Druid, with examples for Druid SQL and Druid's native query format. +This tutorial demonstrates how to query data in Apache Druid using SQL. -The tutorial assumes that you've already completed one of the 4 ingestion tutorials, as we will be querying the sample Wikipedia edits data. +It assumes that you've completed the [Quickstart](../tutorials/index.md) +or one of the following tutorials, since we'll query datasources that you would have created +by following one of them: * [Tutorial: Loading a file](../tutorials/tutorial-batch.md) * [Tutorial: Loading stream data from Kafka](../tutorials/tutorial-kafka.md) * [Tutorial: Loading a file using Hadoop](../tutorials/tutorial-batch-hadoop.md) -Druid queries are sent over HTTP. -The Druid console includes a view to issue queries to Druid and nicely format the results. +There are various ways to run Druid SQL queries: from the Druid console, using a command line utility +and by posting the query by HTTP. We'll look at each of these. -## Druid SQL queries -Druid supports a dialect of SQL for querying. +## Query SQL from the Druid console -This query retrieves the 10 Wikipedia pages with the most page edits on 2015-09-12. +The Druid console includes a view that makes it easier to build and test queries, and +view their results. -```sql -SELECT page, COUNT(*) AS Edits -FROM wikipedia -WHERE TIMESTAMP '2015-09-12 00:00:00' <= "__time" AND "__time" < TIMESTAMP '2015-09-13 00:00:00' -GROUP BY page -ORDER BY Edits DESC -LIMIT 10 -``` +1. Start up the Druid cluster, if it's not already running, and open the Druid console in your web +browser. + +2. Click **Query** from the header to open the Query view: + + ![Query view](../assets/tutorial-query-01.png "Query view") + + You can always write queries directly in the edit pane, but the Query view also provides + facilities to help you construct SQL queries, which we will use to generate a starter query. + +3. Expand the wikipedia datasource tree in the left pane. We'll +create a query for the page dimension. + +4. Click `page` and then **Show:page** from the menu: + + ![Query select page](../assets/tutorial-query-02.png "Query select page") + + A SELECT query appears in the query edit pane and immediately runs. However, in this case, the query + returns no data, since by default the query filters for data from the last day, while our data is considerably + older than that. Let's remove the filter. + +5. In the datasource tree, click `__time` and **Remove Filter**. + + ![Clear WHERE filter](../assets/tutorial-query-03.png "Clear WHERE filter") + +6. Click **Run** to run the query. + + You should now see two columns of data, a page name and the count: + + ![Query results](../assets/tutorial-query-04.png "Query results") + + Notice that the results are limited in the console to about a hundred, by default, due to the **Smart query limit** + feature. This helps users avoid inadvertently running queries that return an excessive amount of data, possibly + overwhelming their system. + +7. Let's edit the query directly and take a look at a few more query building features in the editor. + Click in the query edit pane and make the following changes: + + 1. Add a line after the first column, `"page"` and Start typing the name of a new column, `"countryName"`. Notice that the autocomplete menu suggests column names, functions, keywords, and more. Choose "countryName" and +add the new column to the GROUP BY clause as well, either by name or by reference to its position, `2`. + + 2. For readability, replace `Count` column name with `Edits`, since the `COUNT()` function actually +returns the number of edits for the page. Make the same column name change in the ORDER BY clause as well. + + The `COUNT()` function is one of many functions available for use in Druid SQL queries. You can mouse over a function name + in the autocomplete menu to see a brief description of a function. Also, you can find more information in the Druid + documentation; for example, the `COUNT()` function is documented in + [Aggregation functions](../querying/sql.md#aggregation-functions). + + The query should now be: + + ```sql + SELECT + "page", + "countryName", + COUNT(*) AS "Edits" + FROM "wikipedia" + GROUP BY 1, 2 + ORDER BY "Edits" DESC + ``` + + When you run the query again, notice that we're getting the new dimension,`countryName`, but for most of the rows, its value + is null. Let's + show only rows with a `countryName` value. + +8. Click the countryName dimension in the left pane and choose the first filtering option. It's not exactly what we want, but +we'll edit it by hand. The new WHERE clause should appear in your query. + +8. Modify the WHERE clause to exclude results that do not have a value for countryName: + + ```sql + WHERE "countryName" IS NOT NULL + ``` + Run the query again. You should now see the top edits by country: + + ![Finished query](../assets/tutorial-query-035.png "Finished query") -Let's look at the different ways to issue this query. +9. Under the covers, every Druid SQL query is translated into a query in the JSON-based _Druid native query_ format before it runs + on data nodes. You can view the native query for this query by clicking `...` and **Explain SQL Query**. -### Query SQL via the console + While you can use Druid SQL for most purposes, familiarity with native query is useful for composing complex queries and for troubleshooting +performance issues. For more information, see [Native queries](../querying/querying.md). -You can issue the above query from the console. + ![Explain query](../assets/tutorial-query-06.png "Explain query") -![Query autocomplete](../assets/tutorial-query-01.png "Query autocomplete") + > Another way to view the explain plan is by adding EXPLAIN PLAN FOR to the front of your query, as follows: + > + >```sql + >EXPLAIN PLAN FOR + >SELECT + > "page", + > "countryName", + > COUNT(*) AS "Edits" + >FROM "wikipedia" + >WHERE "countryName" IS NOT NULL + >GROUP BY 1, 2 + >ORDER BY "Edits" DESC + >``` + >This is particularly useful when running queries + from the command line or over HTTP. -The console query view provides autocomplete functionality with inline documentation. -![Query options](../assets/tutorial-query-02.png "Query options") +9. Finally, click `...` and **Edit context** to see how you can add additional parameters controlling the execution of the query execution. In the field, enter query context options as JSON key-value pairs, as described in [Context flags](../querying/query-context.md). -You can also configure extra [context flags](../querying/query-context.md) to be sent with the query from the `...` options menu. +That's it! We've built a simple query using some of the query builder features built into the Druid Console. The following +sections provide a few more example queries you can try. Also, see [Other ways to invoke SQL queries](#other-ways-to-invoke-sql-queries) to learn how +to run Druid SQL from the command line or over HTTP. -Note that the console will (by default) wrap your SQL queries in a limit where appropriate so that queries such as `SELECT * FROM wikipedia` can complete. -You can turn off this behavior from the `Smart query limit` toggle. +## More Druid SQL examples -![Query actions](../assets/tutorial-query-03.png "Query actions") +Here is a collection of queries to try out: + +### Query over time + +```sql +SELECT FLOOR(__time to HOUR) AS HourTime, SUM(deleted) AS LinesDeleted +FROM wikipedia WHERE "__time" BETWEEN TIMESTAMP '2015-09-12 00:00:00' AND TIMESTAMP '2015-09-13 00:00:00' +GROUP BY 1 +``` + +![Query example](../assets/tutorial-query-07.png "Query example") + +### General group by + +```sql +SELECT channel, page, SUM(added) +FROM wikipedia WHERE "__time" BETWEEN TIMESTAMP '2015-09-12 00:00:00' AND TIMESTAMP '2015-09-13 00:00:00' +GROUP BY channel, page +ORDER BY SUM(added) DESC +``` -The query view provides contextual actions that can write and modify the query for you. +![Query example](../assets/tutorial-query-08.png "Query example") + + +## Other ways to invoke SQL queries ### Query SQL via dsql -For convenience, the Druid package includes a SQL command-line client, located at `bin/dsql` from the Druid package root. +For convenience, the Druid package includes a SQL command-line client, located at `bin/dsql` in the Druid package root. Let's now run `bin/dsql`; you should see the following prompt: @@ -107,7 +215,8 @@ Retrieved 10 rows in 0.06s. ### Query SQL over HTTP -The SQL queries are submitted as JSON over HTTP. + +You can submit queries directly to the Druid Broker over HTTP. The tutorial package includes an example file that contains the SQL query shown above at `quickstart/tutorial/wikipedia-top-pages-sql.json`. Let's submit that query to the Druid Broker: @@ -162,150 +271,8 @@ The following results should be returned: ] ``` -### More Druid SQL examples - -Here is a collection of queries to try out: - -#### Query over time - -```sql -SELECT FLOOR(__time to HOUR) AS HourTime, SUM(deleted) AS LinesDeleted -FROM wikipedia WHERE "__time" BETWEEN TIMESTAMP '2015-09-12 00:00:00' AND TIMESTAMP '2015-09-13 00:00:00' -GROUP BY 1 -``` - -![Query example](../assets/tutorial-query-03.png "Query example") - -#### General group by - -```sql -SELECT channel, page, SUM(added) -FROM wikipedia WHERE "__time" BETWEEN TIMESTAMP '2015-09-12 00:00:00' AND TIMESTAMP '2015-09-13 00:00:00' -GROUP BY channel, page -ORDER BY SUM(added) DESC -``` - -![Query example](../assets/tutorial-query-04.png "Query example") - -#### Select raw data - -```sql -SELECT user, page -FROM wikipedia WHERE "__time" BETWEEN TIMESTAMP '2015-09-12 02:00:00' AND TIMESTAMP '2015-09-12 03:00:00' -LIMIT 5 -``` - -![Query example](../assets/tutorial-query-05.png "Query example") - -### Explain query plan - -Druid SQL has the ability to explain the query plan for a given query. -In the console this functionality is accessible from the `...` button. - -![Explain query](../assets/tutorial-query-06.png "Explain query") - -If you are querying in other ways you can get the plan by prepending `EXPLAIN PLAN FOR ` to a Druid SQL query. - -Using a query from an example above: - -`EXPLAIN PLAN FOR SELECT page, COUNT(*) AS Edits FROM wikipedia WHERE "__time" BETWEEN TIMESTAMP '2015-09-12 00:00:00' AND TIMESTAMP '2015-09-13 00:00:00' GROUP BY page ORDER BY Edits DESC LIMIT 10;` - -```bash -dsql> EXPLAIN PLAN FOR SELECT page, COUNT(*) AS Edits FROM wikipedia WHERE "__time" BETWEEN TIMESTAMP '2015-09-12 00:00:00' AND TIMESTAMP '2015-09-13 00:00:00' GROUP BY page ORDER BY Edits DESC LIMIT 10; -┌─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ -│ PLAN │ -├─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┤ -│ DruidQueryRel(query=[{"queryType":"topN","dataSource":{"type":"table","name":"wikipedia"},"virtualColumns":[],"dimension":{"type":"default","dimension":"page","outputName":"d0","outputType":"STRING"},"metric":{"type":"numeric","metric":"a0"},"threshold":10,"intervals":{"type":"intervals","intervals":["2015-09-12T00:00:00.000Z/2015-09-13T00:00:00.001Z"]},"filter":null,"granularity":{"type":"all"},"aggregations":[{"type":"count","name":"a0"}],"postAggregations":[],"context":{},"descending":false}], signature=[{d0:STRING, a0:LONG}]) │ -└─────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ -Retrieved 1 row in 0.03s. -``` - - -## Native JSON queries - -Druid's native query format is expressed in JSON. - -### Native query via the console - -You can issue native Druid queries from the console's Query view. - -Here is a query that retrieves the 10 Wikipedia pages with the most page edits on 2015-09-12. - -```json -{ - "queryType" : "topN", - "dataSource" : "wikipedia", - "intervals" : ["2015-09-12/2015-09-13"], - "granularity" : "all", - "dimension" : "page", - "metric" : "count", - "threshold" : 10, - "aggregations" : [ - { - "type" : "count", - "name" : "count" - } - ] -} -``` - -Simply paste it into the console to switch the editor into JSON mode. - -![Native query](../assets/tutorial-query-07.png "Native query") - - -### Native queries over HTTP - -We have included a sample native TopN query under `quickstart/tutorial/wikipedia-top-pages.json`: - -Let's submit this query to Druid: - -```bash -curl -X 'POST' -H 'Content-Type:application/json' -d @quickstart/tutorial/wikipedia-top-pages.json http://localhost:8888/druid/v2?pretty -``` - -You should see the following query results: - -```json -[ { - "timestamp" : "2015-09-12T00:46:58.771Z", - "result" : [ { - "count" : 33, - "page" : "Wikipedia:Vandalismusmeldung" - }, { - "count" : 28, - "page" : "User:Cyde/List of candidates for speedy deletion/Subpage" - }, { - "count" : 27, - "page" : "Jeremy Corbyn" - }, { - "count" : 21, - "page" : "Wikipedia:Administrators' noticeboard/Incidents" - }, { - "count" : 20, - "page" : "Flavia Pennetta" - }, { - "count" : 18, - "page" : "Total Drama Presents: The Ridonculous Race" - }, { - "count" : 18, - "page" : "User talk:Dudeperson176123" - }, { - "count" : 18, - "page" : "Wikipédia:Le Bistro/12 septembre 2015" - }, { - "count" : 17, - "page" : "Wikipedia:In the news/Candidates" - }, { - "count" : 17, - "page" : "Wikipedia:Requests for page protection" - } ] -} ] -``` - - ## Further reading -The [Queries documentation](../querying/querying.md) has more information on Druid's native JSON queries. +See the [Druid SQL documentation](../querying/sql.md) for more information on using Druid SQL queries. -The [Druid SQL documentation](../querying/sql.md) has more information on using Druid SQL queries. +See the [Queries documentation](../querying/querying.md) for more information on Druid native queries. diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfigTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfigTest.java index 83fad58ade7d..f2ea37a40fc7 100644 --- a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfigTest.java +++ b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfigTest.java @@ -56,7 +56,7 @@ public void testSerDeAmbariMetricsEmitterConfig() throws IOException 1000L, 100, new SendAllTimelineEventConverter("prefix", "druid"), - Collections.EMPTY_LIST, + Collections.emptyList(), 500L, 400L ); diff --git a/extensions-contrib/dropwizard-emitter/src/main/java/org/apache/druid/emitter/dropwizard/DropwizardEmitterModule.java b/extensions-contrib/dropwizard-emitter/src/main/java/org/apache/druid/emitter/dropwizard/DropwizardEmitterModule.java index a700b88050cb..524dad165e38 100644 --- a/extensions-contrib/dropwizard-emitter/src/main/java/org/apache/druid/emitter/dropwizard/DropwizardEmitterModule.java +++ b/extensions-contrib/dropwizard-emitter/src/main/java/org/apache/druid/emitter/dropwizard/DropwizardEmitterModule.java @@ -42,7 +42,7 @@ public class DropwizardEmitterModule implements DruidModule @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java b/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java index bca977211f5b..3f9904529f12 100644 --- a/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java +++ b/extensions-contrib/graphite-emitter/src/main/java/org/apache/druid/emitter/graphite/GraphiteEmitterModule.java @@ -44,7 +44,7 @@ public class GraphiteEmitterModule implements DruidModule @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/GraphiteEmitterConfigTest.java b/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/GraphiteEmitterConfigTest.java index a1ebdaaeafe7..7bde9f3c1627 100644 --- a/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/GraphiteEmitterConfigTest.java +++ b/extensions-contrib/graphite-emitter/src/test/java/org/apache/druid/emitter/graphite/GraphiteEmitterConfigTest.java @@ -53,8 +53,8 @@ public void testSerDeserGraphiteEmitterConfig() throws IOException 1000L, 100, new SendAllGraphiteEventConverter("prefix", true, true, false), - Collections.EMPTY_LIST, - Collections.EMPTY_LIST, + Collections.emptyList(), + Collections.emptyList(), null, null ); @@ -92,4 +92,10 @@ public void testSerDeserDruidToGraphiteEventConverter() throws IOException .readValue(whiteListBasedConverterString); Assert.assertEquals(druidToGraphiteEventConverter, whiteListBasedConverter); } + + @Test + public void testJacksonModules() + { + Assert.assertTrue(new GraphiteEmitterModule().getJacksonModules().isEmpty()); + } } diff --git a/extensions-contrib/influxdb-emitter/src/main/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterModule.java b/extensions-contrib/influxdb-emitter/src/main/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterModule.java index f6e6fa49efad..b286a972c182 100644 --- a/extensions-contrib/influxdb-emitter/src/main/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterModule.java +++ b/extensions-contrib/influxdb-emitter/src/main/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterModule.java @@ -42,7 +42,7 @@ public class InfluxdbEmitterModule implements DruidModule @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/extensions-contrib/influxdb-emitter/src/test/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterTest.java b/extensions-contrib/influxdb-emitter/src/test/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterTest.java index 2095a2f70876..318f38f65c33 100644 --- a/extensions-contrib/influxdb-emitter/src/test/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterTest.java +++ b/extensions-contrib/influxdb-emitter/src/test/java/org/apache/druid/emitter/influxdb/InfluxdbEmitterTest.java @@ -205,4 +205,10 @@ public void testMetricIsInDefaultDimensionWhitelist() String actual = influxdbEmitter.transformForInfluxSystems(event); Assert.assertEquals(expected, actual); } + + @Test + public void testJacksonModules() + { + Assert.assertTrue(new InfluxdbEmitterModule().getJacksonModules().isEmpty()); + } } diff --git a/extensions-contrib/kafka-emitter/pom.xml b/extensions-contrib/kafka-emitter/pom.xml index 8feed6e97121..1f69ae35b333 100644 --- a/extensions-contrib/kafka-emitter/pom.xml +++ b/extensions-contrib/kafka-emitter/pom.xml @@ -38,7 +38,7 @@ org.apache.kafka kafka-clients - 0.10.2.2 + ${apache.kafka.version} org.apache.druid diff --git a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterModule.java b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterModule.java index 795d723d10b6..f83932ed78e9 100644 --- a/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterModule.java +++ b/extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterModule.java @@ -39,7 +39,7 @@ public class KafkaEmitterModule implements DruidModule @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java index 761cbc4160fc..89e75fc28089 100644 --- a/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java +++ b/extensions-contrib/kafka-emitter/src/test/java/org/apache/druid/emitter/kafka/KafkaEmitterConfigTest.java @@ -68,4 +68,10 @@ public void testSerDeNotRequiredKafkaProducerConfig() Assert.fail(); } } + + @Test + public void testJacksonModules() + { + Assert.assertTrue(new KafkaEmitterModule().getJacksonModules().isEmpty()); + } } 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 ec276795ddf7..766f5109269e 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 @@ -140,7 +140,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ), @@ -151,7 +151,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ), @@ -162,7 +162,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -175,7 +175,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ), @@ -186,7 +186,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -209,7 +209,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -225,7 +225,7 @@ public void testCheckSegments() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) @@ -246,7 +246,7 @@ public void testCheckSegmentsAndSubmitTasks() throws IOException ImmutableMap.of(), ImmutableList.of("dim1", "dim2"), ImmutableList.of("m1"), - new HashBasedNumberedShardSpec(0, 1, null, null), + new HashBasedNumberedShardSpec(0, 1, 0, 1, null, null), 9, 1024 ) diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 971f8ed8b264..23dc65f5235c 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.inject.Injector; import com.google.inject.Module; @@ -64,6 +65,7 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.server.ClientQuerySegmentWalker; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.initialization.ServerConfig; @@ -377,6 +379,7 @@ public void emit(Event event) baseClient, null /* local client; unused in this test, so pass in null */, warehouse, + new MapJoinableFactory(ImmutableMap.of()), retryConfig, jsonMapper, serverConfig, diff --git a/extensions-contrib/opentsdb-emitter/src/main/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterModule.java b/extensions-contrib/opentsdb-emitter/src/main/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterModule.java index b107161fe8e0..a46d7150a54f 100644 --- a/extensions-contrib/opentsdb-emitter/src/main/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterModule.java +++ b/extensions-contrib/opentsdb-emitter/src/main/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterModule.java @@ -39,7 +39,7 @@ public class OpentsdbEmitterModule implements DruidModule @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/extensions-contrib/opentsdb-emitter/src/test/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterConfigTest.java b/extensions-contrib/opentsdb-emitter/src/test/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterConfigTest.java index b533877faa32..66c8f62e3320 100644 --- a/extensions-contrib/opentsdb-emitter/src/test/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterConfigTest.java +++ b/extensions-contrib/opentsdb-emitter/src/test/java/org/apache/druid/emitter/opentsdb/OpentsdbEmitterConfigTest.java @@ -76,4 +76,9 @@ public void testSerDeserOpentsdbEmitterConfigWithEmptyNamespacePrefix() throws E Assert.assertEquals(expectedOpentsdbEmitterConfig, opentsdbEmitterConfig); } + @Test + public void testJacksonModules() + { + Assert.assertTrue(new OpentsdbEmitterModule().getJacksonModules().isEmpty()); + } } diff --git a/extensions-contrib/prometheus-emitter/pom.xml b/extensions-contrib/prometheus-emitter/pom.xml new file mode 100644 index 000000000000..ea6864398e5b --- /dev/null +++ b/extensions-contrib/prometheus-emitter/pom.xml @@ -0,0 +1,123 @@ + + + + + druid + org.apache.druid + 0.18.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + + org.apache.druid.extensions.contrib + prometheus-emitter + prometheus-emitter + Extension support for collecting Druid metrics with Prometheus + + + + org.apache.druid + druid-core + ${project.parent.version} + provided + + + io.prometheus + simpleclient + 0.7.0 + + + io.prometheus + simpleclient_httpserver + 0.7.0 + + + io.prometheus + simpleclient_pushgateway + 0.7.0 + + + com.google.code.findbugs + jsr305 + provided + + + com.fasterxml.jackson.core + jackson-annotations + provided + + + joda-time + joda-time + provided + + + com.google.guava + guava + provided + + + com.google.inject + guice + provided + + + com.fasterxml.jackson.core + jackson-databind + provided + + + com.fasterxml.jackson.core + jackson-core + provided + + + junit + junit + test + + + org.easymock + easymock + test + + + pl.pragmatists + JUnitParams + test + + + org.apache.druid + druid-server + ${project.parent.version} + test-jar + test + + + org.apache.druid + druid-processing + ${project.parent.version} + test-jar + test + + + diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.java new file mode 100644 index 000000000000..ede4977aeee6 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/DimensionsAndCollector.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.emitter.prometheus; + +import io.prometheus.client.SimpleCollector; + +public class DimensionsAndCollector +{ + private final String[] dimensions; + private final SimpleCollector collector; + private final double conversionFactor; + + DimensionsAndCollector(String[] dimensions, SimpleCollector collector, double conversionFactor) + { + this.dimensions = dimensions; + this.collector = collector; + this.conversionFactor = conversionFactor; + } + + public String[] getDimensions() + { + return dimensions; + } + + public SimpleCollector getCollector() + { + return collector; + } + + public double getConversionFactor() + { + return conversionFactor; + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java new file mode 100644 index 000000000000..fd62a7b35ccc --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/Metrics.java @@ -0,0 +1,157 @@ +/* + * 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.emitter.prometheus; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Strings; +import io.prometheus.client.Counter; +import io.prometheus.client.Gauge; +import io.prometheus.client.Histogram; +import io.prometheus.client.SimpleCollector; +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 java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.HashMap; +import java.util.Map; +import java.util.SortedSet; +import java.util.regex.Pattern; + +public class Metrics +{ + + private static final Logger log = new Logger(Metrics.class); + private final Map map = new HashMap<>(); + private final ObjectMapper mapper = new ObjectMapper(); + private final Pattern pattern = Pattern.compile("[^a-zA-Z_:][^a-zA-Z0-9_:]*"); + + public DimensionsAndCollector getByName(String name, String service) + { + if (map.containsKey(name)) { + return map.get(name); + } else if (map.containsKey(service + "_" + name)) { + return map.get(service + "_" + name); + } else { + return null; + } + } + + public Metrics(String namespace, String path) + { + Map metrics = readMap(path); + for (Map.Entry entry : metrics.entrySet()) { + String name = entry.getKey(); + Metric metric = entry.getValue(); + Metric.Type type = metric.type; + String[] dimensions = metric.dimensions.toArray(new String[0]); + String formattedName = pattern.matcher(StringUtils.toLowerCase(name)).replaceAll("_"); + SimpleCollector collector = null; + if (Metric.Type.count.equals(type)) { + collector = new Counter.Builder() + .namespace(namespace) + .name(formattedName) + .labelNames(dimensions) + .help(metric.help) + .register(); + } else if (Metric.Type.gauge.equals(type)) { + collector = new Gauge.Builder() + .namespace(namespace) + .name(formattedName) + .labelNames(dimensions) + .help(metric.help) + .register(); + } else if (Metric.Type.timer.equals(type)) { + collector = new Histogram.Builder() + .namespace(namespace) + .name(formattedName) + .labelNames(dimensions) + .buckets(.1, .25, .5, .75, 1, 2.5, 5, 7.5, 10, 30, 60, 120, 300) + .help(metric.help) + .register(); + } else { + log.error("Unrecognized metric type [%s]", type); + } + + if (collector != null) { + map.put(name, new DimensionsAndCollector(dimensions, collector, metric.conversionFactor)); + } + } + + } + + private Map readMap(String path) + { + try { + InputStream is; + if (Strings.isNullOrEmpty(path)) { + log.info("Using default metric dimension and types"); + is = this.getClass().getClassLoader().getResourceAsStream("defaultMetrics.json"); + } else { + log.info("Using metric dimensions at types at [%s]", path); + is = new FileInputStream(new File(path)); + } + return mapper.readerFor(new TypeReference>() + { + }).readValue(is); + } + catch (IOException e) { + throw new ISE(e, "Failed to parse metric dimensions and types"); + } + } + + public Map getMap() + { + return map; + } + + public static class Metric + { + public final SortedSet dimensions; + public final Type type; + public final String help; + public final double conversionFactor; + + @JsonCreator + public Metric( + @JsonProperty("dimensions") SortedSet dimensions, + @JsonProperty("type") Type type, + @JsonProperty("help") String help, + @JsonProperty("conversionFactor") double conversionFactor + ) + { + this.dimensions = dimensions; + this.type = type; + this.help = help; + this.conversionFactor = conversionFactor; + } + + public enum Type + { + count, gauge, timer + } + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java new file mode 100644 index 000000000000..945f46100ed9 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitter.java @@ -0,0 +1,153 @@ +/* + * 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.emitter.prometheus; + + +import com.google.common.collect.ImmutableMap; +import io.prometheus.client.Counter; +import io.prometheus.client.Gauge; +import io.prometheus.client.Histogram; +import io.prometheus.client.exporter.HTTPServer; +import io.prometheus.client.exporter.PushGateway; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.emitter.core.Emitter; +import org.apache.druid.java.util.emitter.core.Event; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; + +import java.io.IOException; +import java.util.Map; +import java.util.regex.Pattern; + +/** + * + */ +public class PrometheusEmitter implements Emitter +{ + + private static final Logger log = new Logger(PrometheusEmitter.class); + private final Metrics metrics; + private final PrometheusEmitterConfig config; + private final PrometheusEmitterConfig.Strategy strategy; + private final Pattern pattern = Pattern.compile("[^a-zA-Z0-9_][^a-zA-Z0-9_]*"); + + private HTTPServer server; + private PushGateway pushGateway; + private String identifier; + + static PrometheusEmitter of(PrometheusEmitterConfig config) + { + return new PrometheusEmitter(config); + } + + public PrometheusEmitter(PrometheusEmitterConfig config) + { + this.config = config; + this.strategy = config.getStrategy(); + metrics = new Metrics(config.getNamespace(), config.getDimensionMapPath()); + } + + + @Override + public void start() + { + if (strategy.equals(PrometheusEmitterConfig.Strategy.exporter)) { + if (server == null) { + try { + server = new HTTPServer(config.getPort()); + } + catch (IOException e) { + log.error(e, "Unable to start prometheus HTTPServer"); + } + } else { + log.error("HTTPServer is already started"); + } + } else if (strategy.equals(PrometheusEmitterConfig.Strategy.pushgateway)) { + pushGateway = new PushGateway(config.getPushGatewayAddress()); + } + + } + + @Override + public void emit(Event event) + { + if (event instanceof ServiceMetricEvent) { + emitMetric((ServiceMetricEvent) event); + } + } + + void emitMetric(ServiceMetricEvent metricEvent) + { + String name = metricEvent.getMetric(); + String service = metricEvent.getService(); + Map userDims = metricEvent.getUserDims(); + identifier = (userDims.get("task") == null ? metricEvent.getHost() : (String) userDims.get("task")); + Number value = metricEvent.getValue(); + + DimensionsAndCollector metric = metrics.getByName(name, service); + if (metric != null) { + String[] labelValues = new String[metric.getDimensions().length]; + String[] labelNames = metric.getDimensions(); + for (int i = 0; i < labelValues.length; i++) { + String labelName = labelNames[i]; + //labelName is controlled by the user. Instead of potential NPE on invalid labelName we use "unknown" as the dimension value + Object userDim = userDims.get(labelName); + labelValues[i] = userDim != null ? pattern.matcher(userDim.toString()).replaceAll("_") : "unknown"; + } + + if (metric.getCollector() instanceof Counter) { + ((Counter) metric.getCollector()).labels(labelValues).inc(value.doubleValue()); + } else if (metric.getCollector() instanceof Gauge) { + ((Gauge) metric.getCollector()).labels(labelValues).set(value.doubleValue()); + } else if (metric.getCollector() instanceof Histogram) { + ((Histogram) metric.getCollector()).labels(labelValues).observe(value.doubleValue() / metric.getConversionFactor()); + } else { + log.error("Unrecognized metric type [%s]", metric.getCollector().getClass()); + } + } else { + log.debug("Unmapped metric [%s]", name); + } + } + + @Override + public void flush() + { + Map map = metrics.getMap(); + try { + for (DimensionsAndCollector collector : map.values()) { + pushGateway.push(collector.getCollector(), config.getNamespace(), ImmutableMap.of(config.getNamespace(), identifier)); + } + } + catch(IOException e){ + log.error(e, "Unable to push prometheus metrics to pushGateway"); + } + } + + @Override + public void close() + { + if (strategy.equals(PrometheusEmitterConfig.Strategy.exporter)) { + if (server != null) { + server.stop(); + } + } else { + flush(); + } + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.java new file mode 100644 index 000000000000..32439f395cb6 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterConfig.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.emitter.prometheus; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; + +import javax.annotation.Nullable; +import javax.xml.ws.BindingType; +import java.util.regex.Pattern; + +/** + * + */ +public class PrometheusEmitterConfig +{ + + Pattern pattern = Pattern.compile("[a-zA-Z_:][a-zA-Z0-9_:]*"); + + @JsonProperty + private final Strategy strategy; + + @JsonProperty + @Nullable + private final String namespace; + + @JsonProperty + @Nullable + private final String dimensionMapPath; + + @JsonProperty + @Nullable + private final Integer port; + + @JsonProperty + @Nullable + private final String pushGatewayAddress; + + @JsonCreator + public PrometheusEmitterConfig( + @JsonProperty("strategy") @Nullable Strategy strategy, + @JsonProperty("namespace") @Nullable String namespace, + @JsonProperty("dimensionMapPath") @Nullable String dimensionMapPath, + @JsonProperty("port") @Nullable Integer port, + @JsonProperty("pushGatewayAddress") @Nullable String pushGatewayAddress + ) + { + + this.strategy = strategy != null ? strategy : Strategy.exporter; + this.namespace = namespace != null ? namespace : "druid"; + Preconditions.checkArgument(pattern.matcher(this.namespace).matches(), "Invalid namespace " + this.namespace); + this.dimensionMapPath = dimensionMapPath; + this.port = port; + this.pushGatewayAddress = pushGatewayAddress; + } + + public String getNamespace() + { + return namespace; + } + + public String getDimensionMapPath() + { + return dimensionMapPath; + } + + public int getPort() + { + return port; + } + + public String getPushGatewayAddress() + { + return pushGatewayAddress; + } + + public Strategy getStrategy() + { + return strategy; + } + + public enum Strategy + { + exporter, pushgateway + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java new file mode 100644 index 000000000000..bb26c4d37d58 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/java/org/apache/druid/emitter/prometheus/PrometheusEmitterModule.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.emitter.prometheus; + + +import com.fasterxml.jackson.databind.Module; +import com.google.inject.Binder; +import com.google.inject.Provides; +import com.google.inject.name.Named; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.initialization.DruidModule; +import org.apache.druid.java.util.emitter.core.Emitter; + +import java.util.Collections; +import java.util.List; + + +/** + * + */ +public class PrometheusEmitterModule implements DruidModule +{ + private static final String EMITTER_TYPE = "prometheus"; + + @Override + public List getJacksonModules() + { + return Collections.EMPTY_LIST; + } + + @Override + public void configure(Binder binder) + { + JsonConfigProvider.bind(binder, "druid.emitter." + EMITTER_TYPE, PrometheusEmitterConfig.class); + } + + @Provides + @ManageLifecycle + @Named(EMITTER_TYPE) + public Emitter getEmitter(PrometheusEmitterConfig config) + { + return PrometheusEmitter.of(config); + } +} diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/prometheus-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule new file mode 100644 index 000000000000..da92fc068bf8 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/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.emitter.prometheus.PrometheusEmitterModule diff --git a/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json new file mode 100644 index 000000000000..1c2063d44333 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/main/resources/defaultMetrics.json @@ -0,0 +1,127 @@ +{ + "query/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to complete a query."}, + "query/bytes" : { "dimensions" : ["dataSource", "type"], "type" : "count", "help": "Number of bytes returned in query response."}, + "query/node/time" : { "dimensions" : ["server"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual historical/realtime processes."}, + "query/node/ttfb" : { "dimensions" : ["server"], "type" : "timer", "help": "Time to first byte. Seconds elapsed until Broker starts receiving the response from individual historical/realtime processes."}, + "query/node/bytes" : { "dimensions" : ["server"], "type" : "count", "help": "Number of bytes returned from querying individual historical/realtime processes."}, + "query/node/backpressure": { "dimensions" : ["server"], "type" : "timer", "help": "Seconds that the channel to this process has spent suspended due to backpressure."}, + + "query/segment/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment. Includes time to page in the segment from disk."}, + "query/wait/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent waiting for a segment to be scanned."}, + "segment/scan/pending" : { "dimensions" : [], "type" : "gauge", "help": "Number of segments in queue waiting to be scanned."}, + "query/segmentAndCache/time" : { "dimensions" : [], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds taken to query individual segment or hit the cache (if it is enabled on the Historical process)."}, + "query/cpu/time" : { "dimensions" : ["dataSource", "type"], "type" : "timer", "conversionFactor": "1000000", "help": "Seconds of CPU time taken to complete a query"}, + + "query/count" : { "dimensions" : [], "type" : "count", "help": "Number of total queries" }, + "query/success/count" : { "dimensions" : [], "type" : "count", "help": "Number of queries successfully processed"}, + "query/failed/count" : { "dimensions" : [], "type" : "count", "help": "Number of failed queries"}, + "query/interrupted/count" : { "dimensions" : [], "type" : "count", "help": "Number of queries interrupted due to cancellation or timeout"}, + + "query/cache/delta/numEntries" : { "dimensions" : [], "type" : "count", "help": "Number of entries in cache"}, + "query/cache/delta/sizeBytes" : { "dimensions" : [], "type" : "count", "help": "Size of cache in bytes."}, + "query/cache/delta/hits" : { "dimensions" : [], "type" : "count", "help": "Number of cache hits."}, + "query/cache/delta/misses" : { "dimensions" : [], "type" : "count", "help": "Number of cache misses."}, + "query/cache/delta/evictions" : { "dimensions" : [], "type" : "count", "help": "Number of cache evictions."}, + "query/cache/delta/hitRate" : { "dimensions" : [], "type" : "count","help": "Cache hit rate."}, + "query/cache/delta/averageBytes" : { "dimensions" : [], "type" : "count", "help": "Average size of record in bytes"}, + "query/cache/delta/timeouts" : { "dimensions" : [], "type" : "count", "help": "Number of cache timeouts"}, + "query/cache/delta/errors" : { "dimensions" : [], "type" : "count", "help": "Number of cache errors."}, + + "query/cache/total/numEntries" : { "dimensions" : [], "type" : "gauge","help": "Total number of entries in cache" }, + "query/cache/total/sizeBytes" : { "dimensions" : [], "type" : "gauge", "help": "Total size of cache in bytes."}, + "query/cache/total/hits" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache hits."}, + "query/cache/total/misses" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache misses." }, + "query/cache/total/evictions" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache evictions."}, + "query/cache/total/hitRate" : { "dimensions" : [], "type" : "gauge", "help": "Total cache hit rate"}, + "query/cache/total/averageBytes" : { "dimensions" : [], "type" : "gauge", "help": "Total average record size in bytes"}, + "query/cache/total/timeouts" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache timeouts"}, + "query/cache/total/errors" : { "dimensions" : [], "type" : "gauge", "help": "Total number of cache errors" }, + + "ingest/events/thrownAway" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because they are outside the windowPeriod."}, + "ingest/events/unparseable" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are unparseable." }, + "ingest/events/duplicate" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events rejected because the events are duplicated."}, + "ingest/events/processed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of events successfully processed per emission period." }, + "ingest/events/messageGap" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Time gap between the data time in event and current system time."}, + "ingest/rows/output" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of Druid rows persisted."}, + "ingest/persists/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of times persist occurred." }, + "ingest/persists/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent doing intermediate persist."}, + "ingest/persists/cpu" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000000000.0, "help": "Cpu time in Seconds spent on doing intermediate persist." }, + "ingest/persists/backPressure" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Seconds spent creating persist tasks and blocking waiting for them to finish." }, + "ingest/persists/failed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of persists that failed." }, + "ingest/handoff/failed" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of handoffs that failed." }, + "ingest/merge/time" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000.0, "help": "Seconds spent merging intermediate segments" }, + "ingest/merge/cpu" : { "dimensions" : ["dataSource"], "type" : "timer", "conversionFactor": 1000000000.0, "help": "Cpu time in Seconds spent on merging intermediate segments."}, + + "ingest/kafka/lag" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Total lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + "ingest/kafka/maxLag" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Max lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + "ingest/kafka/avgLag" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Average lag between the offsets consumed by the Kafka indexing tasks and latest offsets in Kafka brokers across all partitions. Minimum emission period for this metric is a minute."}, + + "task/success/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of successful tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/failed/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of failed tasks per emission period. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/running/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current running tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/pending/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current pending tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/waiting/count" : { "dimensions" : ["dataSource"], "type" : "count", "help": "Number of current waiting tasks. This metric is only available if the TaskCountStatsMonitor module is included."}, + "task/run/time" : { "dimensions" : ["dataSource", "taskType"], "type" : "timer", "conversionFactor": 1000.0, "help": "Milliseconds taken to run a task."}, + + "segment/added/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of new segments created." }, + "segment/moved/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of segments moved/archived via the Move Task." }, + "segment/nuked/bytes" : { "dimensions" : ["dataSource", "taskType"], "type" : "count", "help": "Size in bytes of segments deleted via the Kill Task." }, + + "segment/assigned/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments assigned to be loaded in the cluster."}, + "segment/moved/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments moved in the cluster." }, + "segment/dropped/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being overshadowed." }, + "segment/deleted/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to rules."}, + "segment/unneeded/count" : { "dimensions" : ["tier"], "type" : "count", "help": "Number of segments dropped due to being marked as unused."}, + "segment/unavailable/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "segment/underReplicated/count" : { "dimensions" : ["dataSource", "tier"], "type" : "gauge", "help": "Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries."}, + "segment/cost/raw" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The raw cost of hosting segments."}, + "segment/cost/normalization" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalization of hosting segments."}, + "segment/cost/normalized" : { "dimensions" : ["tier"], "type" : "count", "help": "Used in cost balancing. The normalized cost of hosting segments."}, + "segment/loadQueue/size" : { "dimensions" : ["server"], "type" : "gauge", "help": "Size in bytes of segments to load."}, + "segment/loadQueue/failed" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments that failed to load."}, + "segment/loadQueue/count" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments to load."}, + "segment/dropQueue/count" : { "dimensions" : ["server"], "type" : "gauge", "help": "Number of segments to drop."}, + "segment/size" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Size in bytes of available segments."}, + "segment/overShadowed/count" : { "dimensions" : [], "type" : "gauge", "help": "Number of overShadowed segments."}, + + "segment/max" : { "dimensions" : [], "type" : "gauge", "help": "Maximum byte limit available for segments."}, + "segment/used" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Bytes used for served segments."}, + "segment/usedPercent" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Percentage of space used by served segments."}, + "segment/pendingDelete" : { "dimensions" : [], "type" : "gauge", "help": "On-disk size in bytes of segments that are waiting to be cleared out"}, + + "jvm/pool/committed" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Committed pool."}, + "jvm/pool/init" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Initial pool."}, + "jvm/pool/max" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Max pool."}, + "jvm/pool/used" : { "dimensions" : ["poolKind", "poolName"], "type" : "gauge", "help": "Pool used."}, + "jvm/bufferpool/count" : { "dimensions" : ["bufferpoolName"], "type" : "gauge", "help": "bufferpool count"}, + "jvm/bufferpool/used" : { "dimensions" : ["bufferpoolName"], "type" : "gauge", "help": "bufferpool used"}, + "jvm/bufferpool/capacity" : { "dimensions" : ["bufferpoolName"], "type" : "gauge", "help": "bufferpool capacity"}, + "jvm/mem/init" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Initial memory"}, + "jvm/mem/max" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Max memory"}, + "jvm/mem/used" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Used memory"}, + "jvm/mem/committed" : { "dimensions" : ["memKind"], "type" : "gauge", "help": "Committed memory"}, + "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count", "help": "Garbage collection count"}, + "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "count", "help": "Count of CPU time in Nanoseconds spent on garbage collection. Note: `jvm/gc/cpu` represents the total time over multiple GC cycles; divide by `jvm/gc/count` to get the mean GC time per cycle."}, + + "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge", "help": "Number of events queued in the EventReceiverFirehose's buffer"}, + + "sys/swap/free" : { "dimensions" : [], "type" : "gauge", "help": "Free swap"}, + "sys/swap/max" : { "dimensions" : [], "type" : "gauge", "help": "Max swap"}, + "sys/swap/pageIn" : { "dimensions" : [], "type" : "gauge", "help": "Paged in swap"}, + "sys/swap/pageOut" : { "dimensions" : [], "type" : "gauge", "help": "Paged out swap"}, + "sys/disk/write/count" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Writes to disk."}, + "sys/disk/read/count" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Reads from disk."}, + "sys/disk/write/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes written to disk. Can we used to determine how much paging is occurring with regards to segments."}, + "sys/disk/read/size" : { "dimensions" : ["fsDevName"], "type" : "count", "help": "Bytes read from disk. Can we used to determine how much paging is occurring with regards to segments."}, + "sys/net/write/size" : { "dimensions" : [], "type" : "count", "help": "Bytes written to the network."}, + "sys/net/read/size" : { "dimensions" : [], "type" : "count", "help": "Bytes read from the network."}, + "sys/fs/used" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge", "help": "Filesystem bytes used."}, + "sys/fs/max" : { "dimensions" : ["fsDevName", "fsDirName", "fsTypeName", "fsSysTypeName", "fsOptions"], "type" : "gauge", "help": "Filesystesm bytes max."}, + "sys/mem/used" : { "dimensions" : [], "type" : "gauge", "help": "Memory used."}, + "sys/mem/max" : { "dimensions" : [], "type" : "gauge", "help": "Memory max"}, + "sys/storage/used" : { "dimensions" : ["fsDirName"], "type" : "gauge", "help": "Disk space used."}, + "sys/cpu" : { "dimensions" : ["cpuName", "cpuTime"], "type" : "gauge", "help": "CPU used"}, + + "coordinator_segment/count" : { "dimensions" : ["dataSource"], "type" : "gauge", "help": "Number of served segments."}, + "historical_segment/count" : { "dimensions" : ["dataSource", "tier", "priority"], "type" : "gauge", "help": "Number of served segments."} +} diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java new file mode 100644 index 000000000000..dfd71ba1c95a --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/MetricsTest.java @@ -0,0 +1,41 @@ +/* + * 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.emitter.prometheus; + +import io.prometheus.client.Histogram; +import org.junit.Assert; +import org.junit.Test; + +public class MetricsTest +{ + @Test + public void testMetricsConfiguration() + { + Metrics metrics = new Metrics("test", null); + DimensionsAndCollector dimensionsAndCollector = metrics.getByName("query/time", "historical"); + DimensionsAndCollector d = metrics.getByName("segment/loadQueue/count", "historical"); + Assert.assertNotNull(dimensionsAndCollector); + String[] dimensions = dimensionsAndCollector.getDimensions(); + Assert.assertEquals("dataSource", dimensions[0]); + Assert.assertEquals("type", dimensions[1]); + Assert.assertEquals(1000.0, dimensionsAndCollector.getConversionFactor(), 0.0); + Assert.assertTrue(dimensionsAndCollector.getCollector() instanceof Histogram); + } +} diff --git a/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.java new file mode 100644 index 000000000000..fc2fdd5f7600 --- /dev/null +++ b/extensions-contrib/prometheus-emitter/src/test/java/org/apache/druid/emitter/prometheus/PrometheusEmitterTest.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.emitter.prometheus; + +import com.google.common.collect.ImmutableMap; +import io.prometheus.client.CollectorRegistry; +import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; +import org.junit.Assert; +import org.junit.Test; + + +public class PrometheusEmitterTest +{ + @Test + public void testEmitter() + { + PrometheusEmitterConfig config = new PrometheusEmitterConfig(PrometheusEmitterConfig.Strategy.exporter, null, null, 0, null); + PrometheusEmitter emitter = new PrometheusEmitter(config); + ServiceMetricEvent build = ServiceMetricEvent.builder() + .setDimension("server", "druid-data01.vpc.region") + .build("segment/loadQueue/count", 10) + .build(ImmutableMap.of("service", "historical")); + Assert.assertEquals("historical", build.getService()); + Assert.assertFalse(build.getUserDims().isEmpty()); + emitter.emit(build); + Double count = CollectorRegistry.defaultRegistry.getSampleValue( + "druid_segment_loadqueue_count", new String[]{"server"}, new String[]{"druid_data01_vpc_region"} + ); + Assert.assertEquals(10, count.intValue()); + } +} diff --git a/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitterModule.java b/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitterModule.java index 0af90124f639..50bebdf0c447 100644 --- a/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitterModule.java +++ b/extensions-contrib/statsd-emitter/src/main/java/org/apache/druid/emitter/statsd/StatsDEmitterModule.java @@ -41,7 +41,7 @@ public class StatsDEmitterModule implements DruidModule @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Override diff --git a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json index 2c7817b516a1..8b964270dd74 100644 --- a/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json +++ b/extensions-contrib/statsd-emitter/src/main/resources/defaultMetricDimensions.json @@ -103,7 +103,7 @@ "jvm/gc/count" : { "dimensions" : ["gcName"], "type" : "count" }, "jvm/gc/cpu" : { "dimensions" : ["gcName"], "type" : "count" }, - "ingest/events/buffered" : { "dimensions" : ["serviceName, bufferCapacity"], "type" : "gauge"}, + "ingest/events/buffered" : { "dimensions" : ["serviceName", "bufferCapacity"], "type" : "gauge"}, "sys/swap/free" : { "dimensions" : [], "type" : "gauge"}, "sys/swap/max" : { "dimensions" : [], "type" : "gauge"}, diff --git a/extensions-contrib/statsd-emitter/src/test/java/org/apache/druid/emitter/statsd/StatsDEmitterTest.java b/extensions-contrib/statsd-emitter/src/test/java/org/apache/druid/emitter/statsd/StatsDEmitterTest.java index 360f511eb9dc..ebab54e47a55 100644 --- a/extensions-contrib/statsd-emitter/src/test/java/org/apache/druid/emitter/statsd/StatsDEmitterTest.java +++ b/extensions-contrib/statsd-emitter/src/test/java/org/apache/druid/emitter/statsd/StatsDEmitterTest.java @@ -236,4 +236,10 @@ public void testAlertEvent() Assert.assertEquals(expectedEvent.getTitle(), actualEvent.getTitle()); Assert.assertEquals(expectedEvent.getText(), actualEvent.getText()); } + + @Test + public void testJacksonModules() + { + Assert.assertTrue(new StatsDEmitterModule().getJacksonModules().isEmpty()); + } } diff --git a/extensions-core/datasketches/pom.xml b/extensions-core/datasketches/pom.xml index 48aae4cef47f..4b969818c290 100644 --- a/extensions-core/datasketches/pom.xml +++ b/extensions-core/datasketches/pom.xml @@ -162,6 +162,11 @@ junit test + + joda-time + joda-time + test + org.easymock easymock diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java index a0435430c02c..c645ca724a57 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchEstimateWithErrorBoundsOperatorConversion.java @@ -47,7 +47,7 @@ public class HllSketchEstimateWithErrorBoundsOperatorConversion extends DirectOp .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.INTEGER) .requiredOperands(1) - .returnType(SqlTypeName.OTHER) + .returnTypeNonNull(SqlTypeName.OTHER) .build(); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchToStringOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchToStringOperatorConversion.java index 189b3b7c916e..fe0c56b56adc 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchToStringOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchToStringOperatorConversion.java @@ -44,7 +44,7 @@ public class HllSketchToStringOperatorConversion extends DirectOperatorConversio private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); public HllSketchToStringOperatorConversion() diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchQuantileOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchQuantileOperatorConversion.java index c571be13e403..2387fe8cccb9 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchQuantileOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchQuantileOperatorConversion.java @@ -34,7 +34,7 @@ public class DoublesSketchQuantileOperatorConversion extends DoublesSketchSingle private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) - .returnType(SqlTypeName.DOUBLE) + .returnTypeNonNull(SqlTypeName.DOUBLE) .build(); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchRankOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchRankOperatorConversion.java index ab54cb1441e1..327f757a7182 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchRankOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchRankOperatorConversion.java @@ -34,7 +34,7 @@ public class DoublesSketchRankOperatorConversion extends DoublesSketchSingleArgB private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.NUMERIC) - .returnType(SqlTypeName.DOUBLE) + .returnTypeNonNull(SqlTypeName.DOUBLE) .build(); public DoublesSketchRankOperatorConversion() diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSummaryOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSummaryOperatorConversion.java index 6465ef71f45e..4dd01fd85402 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSummaryOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSummaryOperatorConversion.java @@ -44,7 +44,7 @@ public class DoublesSketchSummaryOperatorConversion extends DirectOperatorConver private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); public DoublesSketchSummaryOperatorConversion() diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java index 6b73390c192f..b7df576e9498 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregator.java @@ -122,7 +122,9 @@ static void updateUnion(Union union, Object update) union.update((long[]) update); } else if (update instanceof List) { for (Object entry : (List) update) { - union.update(entry.toString()); + if (entry != null) { + union.update(entry.toString()); + } } } else { throw new ISE("Illegal type received while theta sketch merging [%s]", update.getClass()); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchEstimateWithErrorBoundsOperatorConversion.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchEstimateWithErrorBoundsOperatorConversion.java index 7ddd74ee80c7..c54f2c59b454 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchEstimateWithErrorBoundsOperatorConversion.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchEstimateWithErrorBoundsOperatorConversion.java @@ -46,7 +46,7 @@ public class ThetaSketchEstimateWithErrorBoundsOperatorConversion extends Direct private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(StringUtils.toUpperCase(FUNCTION_NAME)) .operandTypes(SqlTypeFamily.ANY, SqlTypeFamily.INTEGER) - .returnType(SqlTypeName.OTHER) + .returnTypeNonNull(SqlTypeName.OTHER) .build(); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index 5876726218dc..6f4c8b6a9aaa 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -28,6 +28,7 @@ import org.apache.druid.common.config.NullHandling; 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.PeriodGranularity; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; @@ -47,7 +48,6 @@ import org.apache.druid.query.aggregation.post.ExpressionPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; -import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -75,6 +75,8 @@ import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; +import org.joda.time.DateTimeZone; +import org.joda.time.Period; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -355,46 +357,33 @@ public void testAvgDailyCountDistinctHllSketch() throws Exception Query expected = GroupByQuery.builder() .setDataSource( new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of( - Filtration.eternity()))) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - new ExpressionVirtualColumn( - "v0", - "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - .setDimensions( - Collections.singletonList( - new DefaultDimensionSpec( - "v0", - "d0", - ValueType.LONG - ) - ) - ) - .setAggregatorSpecs( - Collections.singletonList( - new HllSketchBuildAggregatorFactory( - "a0:a", - "cnt", - null, - null, - ROUND - ) - ) - ) - .setPostAggregatorSpecs( - ImmutableList.of( - new FinalizingFieldAccessPostAggregator("a0", "a0:a") - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of( + Filtration.eternity() + ))) + .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) + .aggregators( + Collections.singletonList( + new HllSketchBuildAggregatorFactory( + "a0:a", + "cnt", + null, + null, + ROUND + ) + ) + ) + .postAggregators( + ImmutableList.of( + new FinalizingFieldAccessPostAggregator("a0", "a0:a") + ) + ) + .context(BaseCalciteQueryTest.getTimeseriesContextWithFloorTime( + ImmutableMap.of("skipEmptyBuckets", true, "sqlQueryId", "dummy"), + "d0" + )) + .build() ) ) .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java index 19716d968e6e..3e7d47d8b07d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregationTest.java @@ -35,8 +35,10 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.Query; import org.apache.druid.query.aggregation.AggregationTestHelper; +import org.apache.druid.query.aggregation.Aggregator; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.PostAggregator; +import org.apache.druid.query.aggregation.TestObjectColumnSelector; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; @@ -493,6 +495,25 @@ public void testRelocation() Assert.assertEquals(holders[0].getEstimate(), holders[1].getEstimate(), 0); } + @Test + public void testUpdateUnionWithNullInList() + { + List value = new ArrayList<>(); + value.add("foo"); + value.add(null); + value.add("bar"); + List[] columnValues = new List[]{value}; + final TestObjectColumnSelector selector = new TestObjectColumnSelector(columnValues); + final Aggregator agg = new SketchAggregator(selector, 4096); + agg.aggregate(); + Assert.assertFalse(agg.isNull()); + Assert.assertNotNull(agg.get()); + Assert.assertTrue(agg.get() instanceof SketchHolder); + Assert.assertEquals(2, ((SketchHolder) agg.get()).getEstimate(), 0); + Assert.assertNotNull(((SketchHolder) agg.get()).getSketch()); + Assert.assertEquals(2, ((SketchHolder) agg.get()).getSketch().getEstimate(), 0); + } + private void assertPostAggregatorSerde(PostAggregator agg) throws Exception { Assert.assertEquals( diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java index b41ab60c3409..201380a1b206 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java @@ -28,6 +28,7 @@ import org.apache.druid.common.config.NullHandling; 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.PeriodGranularity; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; @@ -44,7 +45,6 @@ import org.apache.druid.query.aggregation.post.ArithmeticPostAggregator; import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.aggregation.post.FinalizingFieldAccessPostAggregator; -import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; @@ -72,6 +72,8 @@ import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; +import org.joda.time.DateTimeZone; +import org.joda.time.Period; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -351,48 +353,34 @@ public void testAvgDailyCountDistinctThetaSketch() throws Exception Query expected = GroupByQuery.builder() .setDataSource( - new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of( - Filtration.eternity()))) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - new ExpressionVirtualColumn( - "v0", - "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG, - TestExprMacroTable.INSTANCE - ) - ) - .setDimensions( - Collections.singletonList( - new DefaultDimensionSpec( - "v0", - "d0", - ValueType.LONG - ) - ) - ) - .setAggregatorSpecs( - Collections.singletonList( - new SketchMergeAggregatorFactory( - "a0:a", - "cnt", - null, - null, - null, - null - ) - ) - ) - .setPostAggregatorSpecs( - ImmutableList.of( - new FinalizingFieldAccessPostAggregator("a0", "a0:a") - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + new QueryDataSource(Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of( + Filtration.eternity() + ))) + .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) + .aggregators( + Collections.singletonList( + new SketchMergeAggregatorFactory( + "a0:a", + "cnt", + null, + null, + null, + null + ) + ) + ) + .postAggregators( + ImmutableList.of( + new FinalizingFieldAccessPostAggregator("a0", "a0:a") + ) + ) + .context(BaseCalciteQueryTest.getTimeseriesContextWithFloorTime( + ImmutableMap.of("skipEmptyBuckets", true, "sqlQueryId", "dummy"), + "d0" + )) + .build() ) ) .setInterval(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) diff --git a/extensions-core/kafka-extraction-namespace/pom.xml b/extensions-core/kafka-extraction-namespace/pom.xml index a33468a1bff1..0b5f269af427 100644 --- a/extensions-core/kafka-extraction-namespace/pom.xml +++ b/extensions-core/kafka-extraction-namespace/pom.xml @@ -97,11 +97,6 @@ jackson-databind provided - - org.apache.zookeeper - zookeeper - provided - com.google.guava guava @@ -135,6 +130,13 @@ + + org.apache.zookeeper + zookeeper + + 3.5.8 + test + org.apache.kafka kafka_2.12 @@ -176,7 +178,7 @@ org.scala-lang scala-library - 2.12.7 + 2.12.10 test diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index cfc50ff212d7..1f6a64d6e059 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -141,6 +141,13 @@ junit test + + org.apache.zookeeper + zookeeper + + 3.5.8 + test + org.apache.kafka kafka_2.12 @@ -188,7 +195,7 @@ org.scala-lang scala-library - 2.12.7 + 2.12.10 test diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java index 9f7f9977efb2..6bf40866b5ea 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/KafkaIndexTask.java @@ -191,4 +191,10 @@ public String getType() { return TYPE; } + + @Override + public boolean supportsQueries() + { + return true; + } } 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 eaa2f99a1a54..eb41749c33ad 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 @@ -27,6 +27,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.AsyncFunction; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import org.apache.curator.test.TestingCluster; @@ -74,7 +76,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ListenableFutures; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; @@ -344,6 +345,7 @@ public void testRunAfterDataInserted() throws Exception INPUT_FORMAT ) ); + Assert.assertTrue(task.supportsQueries()); final ListenableFuture future = runTask(task); @@ -900,12 +902,12 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception final ListenableFuture normalReplicaFuture = runTask(normalReplica); // Simulating one replica is slower than the other - final ListenableFuture staleReplicaFuture = ListenableFutures.transformAsync( + final ListenableFuture staleReplicaFuture = Futures.transform( taskExec.submit(() -> { Thread.sleep(1000); return staleReplica; }), - this::runTask + (AsyncFunction) this::runTask ); while (normalReplica.getRunner().getStatus() != Status.PAUSED) { 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 79514d268158..9d8635dd39a9 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 @@ -27,10 +27,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import kafka.admin.AdminUtils; -import kafka.admin.BrokerMetadata; -import kafka.admin.RackAwareMode; -import kafka.utils.ZkUtils; import org.apache.curator.test.TestingCluster; import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.impl.DimensionSchema; @@ -89,10 +85,12 @@ import org.apache.druid.server.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.server.metrics.ExceptionCapturingServiceEmitter; import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewPartitions; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.security.JaasUtils; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.easymock.Capture; @@ -111,8 +109,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import scala.Option; -import scala.collection.Seq; import java.io.File; import java.io.IOException; @@ -148,7 +144,6 @@ public class KafkaSupervisorTest extends EasyMockSupport private static String kafkaHost; private static DataSchema dataSchema; private static int topicPostfix; - private static ZkUtils zkUtils; private final int numThreads; @@ -202,8 +197,6 @@ public static void setupClass() throws Exception kafkaHost = StringUtils.format("localhost:%d", kafkaServer.getPort()); dataSchema = getDataSchema(DATASOURCE); - - zkUtils = ZkUtils.apply(zkServer.getConnectString(), 30000, 30000, JaasUtils.isZkSecurityEnabled()); } @Before @@ -237,9 +230,6 @@ public static void tearDownClass() throws IOException zkServer.stop(); zkServer = null; - - zkUtils.close(); - zkUtils = null; } @Test @@ -790,7 +780,7 @@ public void testDontKillTasksWithMismatchedType() throws Exception EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) @@ -938,7 +928,7 @@ public void testRequeueTaskWhenFailed() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) @@ -1042,7 +1032,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception Capture captured = Capture.newInstance(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes(); @@ -1131,7 +1121,7 @@ public void testQueueNextTasksOnSuccess() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(Status.NOT_STARTED)) @@ -1234,7 +1224,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( @@ -1732,7 +1722,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( @@ -1818,7 +1808,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( new KafkaDataSourceMetadata( @@ -2042,7 +2032,7 @@ public void testResetNoTasks() { EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2067,7 +2057,7 @@ public void testResetDataSourceMetadata() throws Exception supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2123,7 +2113,7 @@ public void testResetNoDataSourceMetadata() supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2156,7 +2146,7 @@ public void testGetOffsetFromStorageForPartitionWithResetOffsetAutomatically() t supervisor = getTestableSupervisor(1, 1, true, true, "PT1H", null, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); @@ -2759,7 +2749,7 @@ public void testResetSuspended() { EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2970,7 +2960,7 @@ public void testDoNotKillCompatibleTasks() EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); @@ -3039,7 +3029,7 @@ public void testKillIncompatibleTasks() List existingTasks = ImmutableList.of(task); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); @@ -3242,8 +3232,12 @@ public void testIsTaskCurrent() private void addSomeEvents(int numEventsPerPartition) throws Exception { - //create topic manually - AdminUtils.createTopic(zkUtils, topic, NUM_PARTITIONS, 1, new Properties(), RackAwareMode.Enforced$.MODULE$); + // create topic manually + try (Admin admin = kafkaServer.newAdminClient()) { + admin.createTopics( + Collections.singletonList(new NewTopic(topic, NUM_PARTITIONS, (short) 1)) + ).all().get(); + } try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { kafkaProducer.initTransactions(); @@ -3266,23 +3260,9 @@ private void addSomeEvents(int numEventsPerPartition) throws Exception private void addMoreEvents(int numEventsPerPartition, int num_partitions) throws Exception { - Seq brokerList = AdminUtils.getBrokerMetadatas( - zkUtils, - RackAwareMode.Enforced$.MODULE$, - Option.apply(zkUtils.getSortedBrokerList()) - ); - scala.collection.Map> replicaAssignment = AdminUtils.assignReplicasToBrokers( - brokerList, - num_partitions, - 1, 0, 0 - ); - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK( - zkUtils, - topic, - replicaAssignment, - new Properties(), - true - ); + try (Admin admin = kafkaServer.newAdminClient()) { + admin.createPartitions(Collections.singletonMap(topic, NewPartitions.increaseTo(num_partitions))).all().get(); + } try (final KafkaProducer kafkaProducer = kafkaServer.newProducer()) { kafkaProducer.initTransactions(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java index 0fee75ec9355..9d3f7c500c44 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java +++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/test/TestBroker.java @@ -25,6 +25,7 @@ import org.apache.druid.indexing.kafka.KafkaConsumerConfigs; import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.StringUtils; +import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.common.serialization.ByteArraySerializer; @@ -100,18 +101,30 @@ public int getPort() public KafkaProducer newProducer() { - return new KafkaProducer(producerProperties()); + return new KafkaProducer<>(producerProperties()); + } + + public Admin newAdminClient() + { + return Admin.create(adminClientProperties()); + } + + Map adminClientProperties() + { + final Map props = new HashMap<>(); + commonClientProperties(props); + return props; } public KafkaConsumer newConsumer() { - return new KafkaConsumer(consumerProperties()); + return new KafkaConsumer<>(consumerProperties()); } - public Map producerProperties() + public Map producerProperties() { - final Map props = new HashMap<>(); - props.put("bootstrap.servers", StringUtils.format("localhost:%d", getPort())); + final Map props = new HashMap<>(); + commonClientProperties(props); props.put("key.serializer", ByteArraySerializer.class.getName()); props.put("value.serializer", ByteArraySerializer.class.getName()); props.put("acks", "all"); @@ -120,6 +133,11 @@ public Map producerProperties() return props; } + void commonClientProperties(Map props) + { + props.put("bootstrap.servers", StringUtils.format("localhost:%d", getPort())); + } + public Map consumerProperties() { final Map props = KafkaConsumerConfigs.getConsumerProperties(); diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java index bfd375830497..9eee3bf6274d 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisIndexTask.java @@ -137,6 +137,12 @@ public String getType() return TYPE; } + @Override + public boolean supportsQueries() + { + return true; + } + @VisibleForTesting AWSCredentialsConfig getAwsCredentialsConfig() { diff --git a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java index b32ee149f544..23e5bec573f1 100644 --- a/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java +++ b/extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplier.java @@ -110,7 +110,8 @@ private static boolean isServiceExceptionRecoverable(AmazonServiceException ex) { final boolean isIOException = ex.getCause() instanceof IOException; final boolean isTimeout = "RequestTimeout".equals(ex.getErrorCode()); - return isIOException || isTimeout; + final boolean isInternalError = ex.getStatusCode() == 500 || ex.getStatusCode() == 503; + return isIOException || isTimeout || isInternalError; } /** @@ -560,16 +561,18 @@ public void close() assign(ImmutableSet.of()); - scheduledExec.shutdown(); + if (scheduledExec != null) { + scheduledExec.shutdown(); - try { - if (!scheduledExec.awaitTermination(EXCEPTION_RETRY_DELAY_MS, TimeUnit.MILLISECONDS)) { - scheduledExec.shutdownNow(); + try { + if (!scheduledExec.awaitTermination(EXCEPTION_RETRY_DELAY_MS, TimeUnit.MILLISECONDS)) { + scheduledExec.shutdownNow(); + } + } + catch (InterruptedException e) { + log.warn(e, "InterruptedException while shutting down"); + throw new RuntimeException(e); } - } - catch (InterruptedException e) { - log.warn(e, "InterruptedException while shutting down"); - throw new RuntimeException(e); } this.closed = true; @@ -807,6 +810,10 @@ private String getSequenceNumber(StreamPartition partition, ShardIterato ); return true; } + if (throwable instanceof AmazonServiceException) { + AmazonServiceException ase = (AmazonServiceException) throwable; + return isServiceExceptionRecoverable(ase); + } return false; }, GET_SEQUENCE_NUMBER_RETRY_COUNT 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 4daa2fdc5d9c..48ca434e7fd8 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 @@ -30,6 +30,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.AsyncFunction; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.name.Named; @@ -78,7 +80,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ListenableFutures; 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; @@ -346,8 +347,8 @@ public void testRunAfterDataInserted() throws Exception null, false ) - ); + Assert.assertTrue(task.supportsQueries()); final ListenableFuture future = runTask(task); @@ -2429,12 +2430,12 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception ((TestableKinesisIndexTask) staleReplica).setLocalSupplier(recordSupplier2); final ListenableFuture normalReplicaFuture = runTask(normalReplica); // Simulating one replica is slower than the other - final ListenableFuture staleReplicaFuture = ListenableFutures.transformAsync( + final ListenableFuture staleReplicaFuture = Futures.transform( taskExec.submit(() -> { Thread.sleep(1000); return staleReplica; }), - this::runTask + (AsyncFunction) this::runTask ); while (normalReplica.getRunner().getStatus() != SeekableStreamIndexTaskRunner.Status.PAUSED) { diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java index 6b2f32f6cdda..881750de2a79 100644 --- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java +++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisRecordSupplierTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.kinesis; +import com.amazonaws.AmazonServiceException; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClient; import com.amazonaws.services.kinesis.model.DescribeStreamRequest; @@ -316,6 +317,94 @@ public void testPoll() throws InterruptedException Assert.assertEquals(SHARDS_LAG_MILLIS, recordSupplier.getPartitionResourcesTimeLag()); } + @Test + public void testPollWithKinesisInternalFailure() throws InterruptedException + { + recordsPerFetch = 100; + + EasyMock.expect(kinesis.getShardIterator( + EasyMock.anyObject(), + EasyMock.eq(SHARD_ID0), + EasyMock.anyString(), + EasyMock.anyString() + )).andReturn( + getShardIteratorResult0).anyTimes(); + + EasyMock.expect(kinesis.getShardIterator( + EasyMock.anyObject(), + EasyMock.eq(SHARD_ID1), + EasyMock.anyString(), + EasyMock.anyString() + )).andReturn( + getShardIteratorResult1).anyTimes(); + + EasyMock.expect(getShardIteratorResult0.getShardIterator()).andReturn(SHARD0_ITERATOR).anyTimes(); + EasyMock.expect(getShardIteratorResult1.getShardIterator()).andReturn(SHARD1_ITERATOR).anyTimes(); + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD0_ITERATOR, recordsPerFetch))) + .andReturn(getRecordsResult0) + .anyTimes(); + EasyMock.expect(kinesis.getRecords(generateGetRecordsReq(SHARD1_ITERATOR, recordsPerFetch))) + .andReturn(getRecordsResult1) + .anyTimes(); + AmazonServiceException getException = new AmazonServiceException("InternalFailure"); + getException.setErrorCode("InternalFailure"); + getException.setStatusCode(500); + getException.setServiceName("AmazonKinesis"); + EasyMock.expect(getRecordsResult0.getRecords()).andThrow(getException).once(); + EasyMock.expect(getRecordsResult0.getRecords()).andReturn(SHARD0_RECORDS).once(); + AmazonServiceException getException2 = new AmazonServiceException("InternalFailure"); + getException2.setErrorCode("InternalFailure"); + getException2.setStatusCode(503); + getException2.setServiceName("AmazonKinesis"); + EasyMock.expect(getRecordsResult1.getRecords()).andThrow(getException2).once(); + EasyMock.expect(getRecordsResult1.getRecords()).andReturn(SHARD1_RECORDS).once(); + EasyMock.expect(getRecordsResult0.getNextShardIterator()).andReturn(null).anyTimes(); + EasyMock.expect(getRecordsResult1.getNextShardIterator()).andReturn(null).anyTimes(); + EasyMock.expect(getRecordsResult0.getMillisBehindLatest()).andReturn(SHARD0_LAG_MILLIS).once(); + EasyMock.expect(getRecordsResult0.getMillisBehindLatest()).andReturn(SHARD0_LAG_MILLIS).once(); + EasyMock.expect(getRecordsResult1.getMillisBehindLatest()).andReturn(SHARD1_LAG_MILLIS).once(); + EasyMock.expect(getRecordsResult1.getMillisBehindLatest()).andReturn(SHARD1_LAG_MILLIS).once(); + + replayAll(); + + Set> partitions = ImmutableSet.of( + StreamPartition.of(STREAM, SHARD_ID0), + StreamPartition.of(STREAM, SHARD_ID1) + ); + + + recordSupplier = new KinesisRecordSupplier( + kinesis, + recordsPerFetch, + 0, + 2, + false, + 100, + 5000, + 5000, + 60000, + 100, + true + ); + + recordSupplier.assign(partitions); + recordSupplier.seekToEarliest(partitions); + recordSupplier.start(); + + while (recordSupplier.bufferSize() < 14) { + Thread.sleep(100); + } + + List> polledRecords = cleanRecords(recordSupplier.poll( + POLL_TIMEOUT_MILLIS)); + + verifyAll(); + + Assert.assertEquals(partitions, recordSupplier.getAssignment()); + Assert.assertTrue(polledRecords.containsAll(ALL_RECORDS)); + Assert.assertEquals(SHARDS_LAG_MILLIS, recordSupplier.getPartitionResourcesTimeLag()); + } + @Test public void testSeek() throws InterruptedException 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 f3a21d9b2278..6bd309c8b7d3 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 @@ -721,7 +721,7 @@ public void testDontKillTasksWithMismatchedType() throws Exception EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -815,7 +815,7 @@ public void testKillBadPartitionAssignment() throws Exception EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); @@ -886,7 +886,7 @@ public void testRequeueTaskWhenFailed() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -1134,7 +1134,7 @@ public void testQueueNextTasksOnSuccess() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -1260,7 +1260,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception final Capture firstTasks = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata(null) @@ -1882,7 +1882,7 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( @@ -1964,7 +1964,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( @@ -2072,7 +2072,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn( new KinesisDataSourceMetadata( @@ -2380,7 +2380,7 @@ public void testResetNoTasks() EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2410,7 +2410,7 @@ public void testResetDataSourceMetadata() throws Exception supervisor = getTestableSupervisor(1, 1, true, "PT1H", null, null); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -2538,7 +2538,7 @@ public void testGetOffsetFromStorageForPartitionWithResetOffsetAutomatically() t EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); EasyMock.reset(indexerMetadataStorageCoordinator); @@ -3409,7 +3409,7 @@ public void testResetSuspended() .anyTimes(); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); taskRunner.registerListener(EasyMock.anyObject(TaskRunnerListener.class), EasyMock.anyObject(Executor.class)); replayAll(); @@ -3536,7 +3536,7 @@ public void testDoNotKillCompatibleTasks() EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes(); EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(task)).anyTimes(); @@ -3633,7 +3633,7 @@ public void testKillIncompatibleTasks() EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(existingTasks).anyTimes(); EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes(); EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes(); @@ -3850,7 +3850,7 @@ private List testShardSplitPhaseOne() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -3959,7 +3959,7 @@ private List testShardSplitPhaseTwo(List phaseOneTasks) throws Excep EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -4134,7 +4134,7 @@ private void testShardSplitPhaseThree(List phaseTwoTasks) throws Exception EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -4296,7 +4296,7 @@ private List testShardMergePhaseOne() throws Exception Capture captured = Capture.newInstance(CaptureType.ALL); EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -4416,7 +4416,7 @@ private List testShardMergePhaseTwo(List phaseOneTasks) throws Excep EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) @@ -4570,7 +4570,7 @@ private void testShardMergePhaseThree(List phaseTwoTasks) throws Exception EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes(); EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes(); - EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes(); + EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes(); EasyMock.expect(taskClient.getStatusAsync(EasyMock.anyString())) .andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.NOT_STARTED)) diff --git a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java index cb31eb404f05..c276b742b101 100644 --- a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java +++ b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/PollingLookupTest.java @@ -177,7 +177,7 @@ public void testUnapply() ); Assert.assertEquals( "reverse lookup of none existing value should be empty list", - Collections.EMPTY_LIST, + Collections.emptyList(), pollingLookup.unapply("does't exist") ); } diff --git a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/cache/loading/LoadingCacheTest.java b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/cache/loading/LoadingCacheTest.java index f0fc6aff287e..61c81e719ebf 100644 --- a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/cache/loading/LoadingCacheTest.java +++ b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/cache/loading/LoadingCacheTest.java @@ -141,7 +141,7 @@ public Object call() return "value2"; } }); - Assert.assertEquals(loadingCache.getAllPresent(IMMUTABLE_MAP.keySet()), Collections.EMPTY_MAP); + Assert.assertEquals(loadingCache.getAllPresent(IMMUTABLE_MAP.keySet()), Collections.emptyMap()); } @Test diff --git a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/jdbc/JdbcDataFetcherTest.java b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/jdbc/JdbcDataFetcherTest.java index 7fc50e484238..9cb363052ca4 100644 --- a/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/jdbc/JdbcDataFetcherTest.java +++ b/extensions-core/lookups-cached-single/src/test/java/org/apache/druid/server/lookup/jdbc/JdbcDataFetcherTest.java @@ -147,7 +147,7 @@ public void testReverseFetch() ); Assert.assertEquals( "reverse lookup of none existing value should be empty list", - Collections.EMPTY_LIST, + Collections.emptyList(), jdbcDataFetcher.reverseFetchKeys("does't exist") ); } diff --git a/hooks/pre-push.sh b/hooks/pre-push.sh new file mode 100755 index 000000000000..a0928db75baa --- /dev/null +++ b/hooks/pre-push.sh @@ -0,0 +1,17 @@ +#!/bin/bash -eu +# 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. + +mvn checkstyle:checkstyle --fail-at-end diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java index 9f20f171fb7b..62a13d3f9fc4 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DetermineHashedPartitionsJob.java @@ -194,6 +194,8 @@ public boolean run() actualSpecs.add( new HadoopyShardSpec( new HashBasedNumberedShardSpec( + i, + numberOfShards, i, numberOfShards, null, diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java index d9e6a14ce56a..0b246be384b6 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/DeterminePartitionsJob.java @@ -666,7 +666,10 @@ protected void innerReduce(Context context, SortableBytes keyBytes, Iterable constructShardSpecFromShardInfo(String partitionType, Ob if ("hashed".equals(partitionType)) { for (Integer[] shardInfo : (Integer[][]) shardInfoForEachShard) { specs.add(new HashBasedNumberedShardSpec( + shardInfo[0], + shardInfo[1], shardInfo[0], shardInfo[1], null, @@ -573,7 +575,8 @@ private List constructShardSpecFromShardInfo(String partitionType, Ob "host", shardInfo[0], shardInfo[1], - partitionNum++ + partitionNum++, + shardInfoForEachShard.length )); } } else { @@ -693,12 +696,12 @@ private void verifyJob(IndexGeneratorJob job) throws IOException if (forceExtendableShardSpecs) { NumberedShardSpec spec = (NumberedShardSpec) dataSegment.getShardSpec(); Assert.assertEquals(i, spec.getPartitionNum()); - Assert.assertEquals(shardInfo.length, spec.getPartitions()); + Assert.assertEquals(shardInfo.length, spec.getNumCorePartitions()); } else if ("hashed".equals(partitionType)) { Integer[] hashShardInfo = (Integer[]) shardInfo[i]; HashBasedNumberedShardSpec spec = (HashBasedNumberedShardSpec) dataSegment.getShardSpec(); Assert.assertEquals((int) hashShardInfo[0], spec.getPartitionNum()); - Assert.assertEquals((int) hashShardInfo[1], spec.getPartitions()); + Assert.assertEquals((int) hashShardInfo[1], spec.getNumCorePartitions()); } else if ("single".equals(partitionType)) { String[] singleDimensionShardInfo = (String[]) shardInfo[i]; SingleDimensionShardSpec spec = (SingleDimensionShardSpec) dataSegment.getShardSpec(); diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java index 3eb03b6807ff..b99468b13969 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/InputRowSerdeTest.java @@ -147,7 +147,7 @@ public void testSerde() Assert.assertEquals(timestamp, out.getTimestampFromEpoch()); Assert.assertEquals(dims, out.getDimensions()); - Assert.assertEquals(Collections.EMPTY_LIST, out.getDimension("dim_non_existing")); + Assert.assertEquals(Collections.emptyList(), out.getDimension("dim_non_existing")); Assert.assertEquals(ImmutableList.of("d1v"), out.getDimension("d1")); Assert.assertEquals(ImmutableList.of("d2v1", "d2v2"), out.getDimension("d2")); Assert.assertEquals(200L, out.getRaw("d3")); diff --git a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceInputSourceModule.java b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceInputSourceModule.java index 56159e1c1000..6c37d97943b1 100644 --- a/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceInputSourceModule.java +++ b/indexing-service/src/main/java/org/apache/druid/guice/IndexingServiceInputSourceModule.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import org.apache.druid.indexing.input.DruidInputSource; +import org.apache.druid.indexing.input.GeneratorInputSource; import org.apache.druid.initialization.DruidModule; import java.util.List; @@ -37,7 +38,8 @@ public List getJacksonModules() return ImmutableList.of( new SimpleModule("IndexingServiceInputSourceModule") .registerSubtypes( - new NamedType(DruidInputSource.class, "druid") + new NamedType(DruidInputSource.class, "druid"), + new NamedType(GeneratorInputSource.class, "generator") ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java index 40745bfe32f5..728f3dedfd0d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractTask.java @@ -145,6 +145,12 @@ public QueryRunner getQueryRunner(Query query) return null; } + @Override + public boolean supportsQueries() + { + return false; + } + @Override public String getClasspathPrefix() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 1f9865d69d74..9a786fa36ea8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -29,6 +29,7 @@ import com.google.common.base.Supplier; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.AsyncFunction; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.commons.io.FileUtils; @@ -65,7 +66,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.ListenableFutures; import org.apache.druid.java.util.common.guava.CloseQuietly; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -245,6 +245,12 @@ public QueryRunner getQueryRunner(Query query) return (queryPlus, responseContext) -> queryPlus.run(appenderator, responseContext); } + @Override + public boolean supportsQueries() + { + return true; + } + @Override public boolean isReady(TaskActionClient taskActionClient) { @@ -682,7 +688,10 @@ private void publishSegments( committerSupplier.get(), Collections.singletonList(sequenceName) ); - pendingHandoffs.add(ListenableFutures.transformAsync(publishFuture, driver::registerHandoff)); + pendingHandoffs.add(Futures.transform( + publishFuture, + (AsyncFunction) driver::registerHandoff + )); } private void waitForSegmentPublishAndHandoff(long timeout) throws InterruptedException, ExecutionException, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java index 05f0a77a4f47..0ad2e99435a5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingLocalSegmentAllocator.java @@ -20,22 +20,26 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.SurrogateAction; import org.apache.druid.indexing.common.actions.TaskAction; -import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs; import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess; +import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -46,73 +50,70 @@ /** * Allocates all necessary segments locally at the beginning and reuses them. */ -public class CachingLocalSegmentAllocator implements CachingSegmentAllocator +public class CachingLocalSegmentAllocator implements SegmentAllocatorForBatch { - private final String taskId; - private final Map sequenceNameToSegmentId; - private final ShardSpecs shardSpecs; + private final String dataSource; + private final Map> sequenceNameToBucket; + private final Function versionFinder; + private final NonLinearlyPartitionedSequenceNameFunction sequenceNameFunction; + private final boolean isParallel; - @FunctionalInterface - interface IntervalToSegmentIdsCreator - { - /** - * @param versionFinder Returns the version for the specified interval - * - * @return Information for segment preallocation - */ - Map> create( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ); - } + private final Map sequenceNameToSegmentId = new HashMap<>(); + private final Object2IntMap intervalToNextPartitionId = new Object2IntOpenHashMap<>(); CachingLocalSegmentAllocator( TaskToolbox toolbox, String dataSource, String taskId, + GranularitySpec granularitySpec, @Nullable SupervisorTaskAccess supervisorTaskAccess, - IntervalToSegmentIdsCreator intervalToSegmentIdsCreator + CompletePartitionAnalysis partitionAnalysis ) throws IOException { - this.taskId = taskId; - this.sequenceNameToSegmentId = new HashMap<>(); + this.dataSource = dataSource; + this.sequenceNameToBucket = new HashMap<>(); final TaskAction> action; if (supervisorTaskAccess == null) { action = new LockListAction(); + isParallel = false; } else { action = new SurrogateAction<>(supervisorTaskAccess.getSupervisorTaskId(), new LockListAction()); + isParallel = true; } - final Map intervalToVersion = - toolbox.getTaskActionClient() - .submit(action) - .stream() - .collect(Collectors.toMap( - TaskLock::getInterval, - TaskLock::getVersion - )); - Function versionFinder = interval -> findVersion(intervalToVersion, interval); - - final Map> intervalToIds = intervalToSegmentIdsCreator.create( - toolbox, - dataSource, - versionFinder + this.versionFinder = createVersionFinder(toolbox, action); + final Map>> intervalToShardSpecs = partitionAnalysis.createBuckets( + toolbox + ); + + sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( + taskId, + new ShardSpecs(intervalToShardSpecs, granularitySpec.getQueryGranularity()) ); - final Map> shardSpecMap = new HashMap<>(); - for (Entry> entry : intervalToIds.entrySet()) { + for (Entry>> entry : intervalToShardSpecs.entrySet()) { final Interval interval = entry.getKey(); - final List idsPerInterval = intervalToIds.get(interval); + final List> buckets = entry.getValue(); - for (SegmentIdWithShardSpec segmentIdentifier : idsPerInterval) { - shardSpecMap.computeIfAbsent(interval, k -> new ArrayList<>()).add(segmentIdentifier.getShardSpec()); - // The shardSpecs for partitioning and publishing can be different if isExtendableShardSpecs = true. - sequenceNameToSegmentId.put(getSequenceName(interval, segmentIdentifier.getShardSpec()), segmentIdentifier); - } + buckets.forEach(bucket -> { + sequenceNameToBucket.put(sequenceNameFunction.getSequenceName(interval, bucket), Pair.of(interval, bucket)); + }); } - shardSpecs = new ShardSpecs(shardSpecMap); + } + + static Function createVersionFinder( + TaskToolbox toolbox, + TaskAction> lockListAction + ) throws IOException + { + final Map intervalToVersion = + toolbox.getTaskActionClient() + .submit(lockListAction) + .stream() + .collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion)); + + return interval -> findVersion(intervalToVersion, interval); } private static String findVersion(Map intervalToVersion, Interval interval) @@ -132,28 +133,36 @@ public SegmentIdWithShardSpec allocate( boolean skipSegmentLineageCheck ) { - return Preconditions.checkNotNull( - sequenceNameToSegmentId.get(sequenceName), - "Missing segmentId for the sequence[%s]", - sequenceName + return sequenceNameToSegmentId.computeIfAbsent( + sequenceName, + k -> { + final Pair pair = Preconditions.checkNotNull( + sequenceNameToBucket.get(sequenceName), + "Missing bucket for sequence[%s]", + sequenceName + ); + final Interval interval = pair.lhs; + // Determines the partitionId if this segment allocator is used by the single-threaded task. + // In parallel ingestion, the partitionId is determined in the supervisor task. + // See ParallelIndexSupervisorTask.groupGenericPartitionLocationsPerPartition(). + // This code... isn't pretty, but should be simple enough to understand. + final ShardSpec shardSpec = isParallel + ? pair.rhs + : pair.rhs.convert( + intervalToNextPartitionId.computeInt( + interval, + (i, nextPartitionId) -> nextPartitionId == null ? 0 : nextPartitionId + 1 + ) + ); + final String version = versionFinder.apply(interval); + return new SegmentIdWithShardSpec(dataSource, interval, version, shardSpec); + } ); } - /** - * Create a sequence name from the given shardSpec and interval. - * - * See {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec} as an example of partitioning. - */ - private String getSequenceName(Interval interval, ShardSpec shardSpec) - { - // Note: We do not use String format here since this can be called in a tight loop - // and it's faster to add strings together than it is to use String#format - return taskId + "_" + interval + "_" + shardSpec.getPartitionNum(); - } - @Override - public ShardSpecs getShardSpecs() + public SequenceNameFunction getSequenceNameFunction() { - return shardSpecs; + return sequenceNameFunction; } } 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 dac3bd9a0952..59c9019e6397 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 @@ -61,6 +61,7 @@ import org.apache.druid.indexing.input.DruidInputSource; import org.apache.druid.indexing.overlord.Segments; 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.java.util.common.Pair; import org.apache.druid.java.util.common.RE; @@ -98,11 +99,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.TreeMap; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -524,10 +523,30 @@ static List createIngestionSchema( .add(p) ); - final List specs = new ArrayList<>(intervalToSegments.size()); - for (Entry>> entry : intervalToSegments.entrySet()) { - final Interval interval = entry.getKey(); - final List> segmentsToCompact = entry.getValue(); + // unify overlapping intervals to ensure overlapping segments compacting in the same indexSpec + List>>> intervalToSegmentsUnified = new ArrayList<>(); + Interval union = null; + List> segments = new ArrayList<>(); + for (Map.Entry>> entry : intervalToSegments.entrySet()) { + Interval cur = entry.getKey(); + if (union == null) { + union = cur; + segments.addAll(entry.getValue()); + } else if (union.overlaps(cur)) { + union = Intervals.utc(union.getStartMillis(), Math.max(union.getEndMillis(), cur.getEndMillis())); + segments.addAll(entry.getValue()); + } else { + intervalToSegmentsUnified.add(Pair.of(union, segments)); + union = cur; + segments = new ArrayList<>(entry.getValue()); + } + } + intervalToSegmentsUnified.add(Pair.of(union, segments)); + + final List specs = new ArrayList<>(intervalToSegmentsUnified.size()); + for (Pair>> entry : intervalToSegmentsUnified) { + final Interval interval = entry.lhs; + final List> segmentsToCompact = entry.rhs; final DataSchema dataSchema = createDataSchema( segmentProvider.dataSource, segmentsToCompact, @@ -710,20 +729,8 @@ private static DimensionsSpec createDimensionsSpec(List intervalComparator = Comparators.intervalsByStartThenEnd(); - for (int i = 0; i < queryableIndices.size() - 1; i++) { - final Interval shouldBeSmaller = queryableIndices.get(i).lhs.getDataInterval(); - final Interval shouldBeLarger = queryableIndices.get(i + 1).lhs.getDataInterval(); - Preconditions.checkState( - intervalComparator.compare(shouldBeSmaller, shouldBeLarger) <= 0, - "QueryableIndexes are not sorted! Interval[%s] of segment[%s] is laster than interval[%s] of segment[%s]", - shouldBeSmaller, - queryableIndices.get(i).rhs.getId(), - shouldBeLarger, - queryableIndices.get(i + 1).rhs.getId() - ); - } + // sort timelineSegments in order of interval, see https://github.com/apache/druid/pull/9905 + queryableIndices.sort((o1, o2) -> Comparators.intervalsByStartThenEnd().compare(o1.rhs.getInterval(), o2.rhs.getInterval())); int index = 0; for (Pair pair : Lists.reverse(queryableIndices)) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index 712340f4fd70..89d69604be26 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -97,7 +97,6 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver; import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata; import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher; import org.apache.druid.segment.realtime.firehose.ChatHandler; @@ -108,7 +107,6 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.utils.CircularBuffer; import org.codehaus.plexus.util.FileUtils; import org.joda.time.Interval; @@ -878,34 +876,33 @@ private TaskStatus generateAndPublishSegments( final IndexTuningConfig tuningConfig = ingestionSchema.getTuningConfig(); final long pushTimeout = tuningConfig.getPushTimeout(); - final SegmentAllocator segmentAllocator; + final SegmentAllocatorForBatch segmentAllocator; final SequenceNameFunction sequenceNameFunction; switch (partitionsSpec.getType()) { case HASH: case RANGE: - final CachingSegmentAllocator localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning( + final SegmentAllocatorForBatch localSegmentAllocator = SegmentAllocators.forNonLinearPartitioning( toolbox, getDataSource(), getId(), + dataSchema.getGranularitySpec(), null, (CompletePartitionAnalysis) partitionAnalysis ); - sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( - getId(), - localSegmentAllocator.getShardSpecs() - ); + sequenceNameFunction = localSegmentAllocator.getSequenceNameFunction(); segmentAllocator = localSegmentAllocator; break; case LINEAR: segmentAllocator = SegmentAllocators.forLinearPartitioning( toolbox, + getId(), null, dataSchema, getTaskLockHelper(), ingestionSchema.getIOConfig().isAppendToExisting(), partitionAnalysis.getPartitionsSpec() ); - sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(getId()); + sequenceNameFunction = segmentAllocator.getSequenceNameFunction(); break; default: throw new UOE("[%s] secondary partition type is not supported", partitionsSpec.getType()); @@ -1013,36 +1010,6 @@ private static SegmentsAndCommitMetadata awaitPublish( } } - /** - * This class represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. - */ - static class ShardSpecs - { - private final Map> map; - - ShardSpecs(final Map> map) - { - this.map = map; - } - - /** - * Return a shardSpec for the given interval and input row. - * - * @param interval interval for shardSpec - * @param row input row - * - * @return a shardSpec - */ - ShardSpec getShardSpec(Interval interval, InputRow row) - { - final List shardSpecs = map.get(interval); - if (shardSpecs == null || shardSpecs.isEmpty()) { - throw new ISE("Failed to get shardSpec for interval[%s]", interval); - } - return shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(row.getTimestampFromEpoch(), row); - } - } - private static InputFormat getInputFormat(IndexIngestionSpec ingestionSchema) { return ingestionSchema.getIOConfig().getNonNullInputFormat(); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java index ead0f635599f..c2488e6244ba 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/LocalSegmentAllocator.java @@ -30,7 +30,7 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -43,11 +43,12 @@ /** * Segment allocator which allocates new segments locally per request. */ -class LocalSegmentAllocator implements SegmentAllocator +class LocalSegmentAllocator implements SegmentAllocatorForBatch { private final SegmentAllocator internalAllocator; + private final SequenceNameFunction sequenceNameFunction; - LocalSegmentAllocator(TaskToolbox toolbox, String dataSource, GranularitySpec granularitySpec) throws IOException + LocalSegmentAllocator(TaskToolbox toolbox, String taskId, String dataSource, GranularitySpec granularitySpec) throws IOException { final Map intervalToVersion = toolbox .getTaskActionClient() @@ -77,9 +78,10 @@ class LocalSegmentAllocator implements SegmentAllocator dataSource, interval, version, - new NumberedShardSpec(partitionId, 0) + new BuildingNumberedShardSpec(partitionId) ); }; + sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(taskId); } @Nullable @@ -93,4 +95,10 @@ public SegmentIdWithShardSpec allocate( { return internalAllocator.allocate(row, sequenceName, previousSegmentId, skipSegmentLineageCheck); } + + @Override + public SequenceNameFunction getSequenceNameFunction() + { + return sequenceNameFunction; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java index 3031a7a736bc..44fd520f913b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NonLinearlyPartitionedSequenceNameFunction.java @@ -20,8 +20,7 @@ package org.apache.druid.indexing.common.task; import org.apache.druid.data.input.InputRow; -import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs; -import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.joda.time.Interval; /** @@ -31,7 +30,6 @@ * Note that all segment IDs should be allocated upfront to use this function. * * @see org.apache.druid.indexer.partitions.SecondaryPartitionType - * @see CachingSegmentAllocator */ public class NonLinearlyPartitionedSequenceNameFunction implements SequenceNameFunction { @@ -56,10 +54,10 @@ public String getSequenceName(Interval interval, InputRow inputRow) * * See {@link org.apache.druid.timeline.partition.HashBasedNumberedShardSpec} as an example of partitioning. */ - private String getSequenceName(Interval interval, ShardSpec shardSpec) + public String getSequenceName(Interval interval, BucketNumberedShardSpec bucket) { // Note: We do not use String format here since this can be called in a tight loop // and it's faster to add strings together than it is to use String#format - return taskId + "_" + interval + "_" + shardSpec.getPartitionNum(); + return taskId + "_" + interval + "_" + bucket.getBucketId(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java index 1598dee0c301..87daaa865506 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/OverlordCoordinatingSegmentAllocator.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.partition.NumberedOverwritePartialShardSpec; import org.apache.druid.timeline.partition.NumberedPartialShardSpec; @@ -44,12 +43,14 @@ /** * Segment allocator which allocates new segments using the overlord per request. */ -public class OverlordCoordinatingSegmentAllocator implements SegmentAllocator +public class OverlordCoordinatingSegmentAllocator implements SegmentAllocatorForBatch { private final ActionBasedSegmentAllocator internalAllocator; + private final LinearlyPartitionedSequenceNameFunction sequenceNameFunction; OverlordCoordinatingSegmentAllocator( final TaskToolbox toolbox, + final String taskId, final @Nullable SupervisorTaskAccess supervisorTaskAccess, final DataSchema dataSchema, final TaskLockHelper taskLockHelper, @@ -101,6 +102,7 @@ public class OverlordCoordinatingSegmentAllocator implements SegmentAllocator } } ); + this.sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(taskId); } @Nullable @@ -146,4 +148,10 @@ private static PartialShardSpec createPartialShardSpec( ); } } + + @Override + public SequenceNameFunction getSequenceNameFunction() + { + return sequenceNameFunction; + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index ed2ddd2c604e..055a3fea3be7 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -196,6 +196,12 @@ public QueryRunner getQueryRunner(Query query) return plumber.getQueryRunner(query); } + @Override + public boolean supportsQueries() + { + return true; + } + @Override public boolean isReady(TaskActionClient taskActionClient) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocatorForBatch.java similarity index 55% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocatorForBatch.java index 86ae307a196c..f2bf5030a87f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CachingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocatorForBatch.java @@ -19,21 +19,16 @@ package org.apache.druid.indexing.common.task; -import org.apache.druid.indexing.common.task.IndexTask.ShardSpecs; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; /** - * SegmentAllocator that allocates all necessary segments upfront. This allocator should be used for the hash or range - * secondary partitioning. * - * In the hash or range secondary partitioning, the information about all partition buckets should be known before - * the task starts to allocate segments. For example, for the hash partitioning, the task should know how many hash - * buckets it will create, what is the hash value allocated for each bucket, etc. Similar for the range partitioning. */ -public interface CachingSegmentAllocator extends SegmentAllocator +public interface SegmentAllocatorForBatch extends SegmentAllocator { /** - * Returns the {@link org.apache.druid.timeline.partition.ShardSpec}s of all segments allocated upfront. + * + * @return */ - ShardSpecs getShardSpecs(); + SequenceNameFunction getSequenceNameFunction(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java index 3a6b1f51686f..47df3f235b7b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentAllocators.java @@ -24,6 +24,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.SupervisorTaskAccess; import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis; import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import javax.annotation.Nullable; @@ -35,8 +36,9 @@ public final class SegmentAllocators * Creates a new {@link SegmentAllocator} for the linear partitioning. * supervisorTaskAccess can be null if this method is called by the {@link IndexTask}. */ - public static SegmentAllocator forLinearPartitioning( + public static SegmentAllocatorForBatch forLinearPartitioning( final TaskToolbox toolbox, + final String taskId, final @Nullable SupervisorTaskAccess supervisorTaskAccess, final DataSchema dataSchema, final TaskLockHelper taskLockHelper, @@ -47,6 +49,7 @@ public static SegmentAllocator forLinearPartitioning( if (appendToExisting || taskLockHelper.isUseSegmentLock()) { return new OverlordCoordinatingSegmentAllocator( toolbox, + taskId, supervisorTaskAccess, dataSchema, taskLockHelper, @@ -57,12 +60,14 @@ public static SegmentAllocator forLinearPartitioning( if (supervisorTaskAccess == null) { return new LocalSegmentAllocator( toolbox, + taskId, dataSchema.getDataSource(), dataSchema.getGranularitySpec() ); } else { return new SupervisorTaskCoordinatingSegmentAllocator( supervisorTaskAccess.getSupervisorTaskId(), + taskId, supervisorTaskAccess.getTaskClient() ); } @@ -73,10 +78,11 @@ public static SegmentAllocator forLinearPartitioning( * Creates a new {@link SegmentAllocator} for the hash and range partitioning. * supervisorTaskAccess can be null if this method is called by the {@link IndexTask}. */ - public static CachingSegmentAllocator forNonLinearPartitioning( + public static SegmentAllocatorForBatch forNonLinearPartitioning( final TaskToolbox toolbox, final String dataSource, final String taskId, + final GranularitySpec granularitySpec, final @Nullable SupervisorTaskAccess supervisorTaskAccess, final CompletePartitionAnalysis partitionAnalysis ) throws IOException @@ -85,8 +91,9 @@ public static CachingSegmentAllocator forNonLinearPartitioning( toolbox, dataSource, taskId, + granularitySpec, supervisorTaskAccess, - partitionAnalysis::convertToIntervalToSegmentIds + partitionAnalysis ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java new file mode 100644 index 000000000000..3db4beba2735 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ShardSpecs.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.common.task; + +import org.apache.druid.data.input.InputRow; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.granularity.Granularity; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; +import org.joda.time.Interval; + +import java.util.List; +import java.util.Map; + +/** + * This class represents a map of (Interval, ShardSpec) and is used for easy shardSpec generation. + */ +public class ShardSpecs +{ + private final Map>> map; + private final Granularity queryGranularity; + + ShardSpecs(final Map>> map, Granularity queryGranularity) + { + this.map = map; + this.queryGranularity = queryGranularity; + } + + /** + * Return a shardSpec for the given interval and input row. + * + * @param interval interval for shardSpec + * @param row input row + * + * @return a shardSpec + */ + BucketNumberedShardSpec getShardSpec(Interval interval, InputRow row) + { + final List> shardSpecs = map.get(interval); + if (shardSpecs == null || shardSpecs.isEmpty()) { + throw new ISE("Failed to get shardSpec for interval[%s]", interval); + } + final long truncatedTimestamp = queryGranularity.bucketStart(row.getTimestamp()).getMillis(); + return (BucketNumberedShardSpec) shardSpecs.get(0).getLookup(shardSpecs).getShardSpec(truncatedTimestamp, row); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java index 7fde4b879d00..d0956aadbc8f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SupervisorTaskCoordinatingSegmentAllocator.java @@ -21,7 +21,6 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTaskClient; -import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import java.io.IOException; @@ -29,18 +28,21 @@ /** * Segment allocator that allocates new segments using the supervisor task per request. */ -public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAllocator +public class SupervisorTaskCoordinatingSegmentAllocator implements SegmentAllocatorForBatch { private final String supervisorTaskId; private final ParallelIndexSupervisorTaskClient taskClient; + private final SequenceNameFunction sequenceNameFunction; SupervisorTaskCoordinatingSegmentAllocator( String supervisorTaskId, + String taskId, ParallelIndexSupervisorTaskClient taskClient ) { this.supervisorTaskId = supervisorTaskId; this.taskClient = taskClient; + this.sequenceNameFunction = new LinearlyPartitionedSequenceNameFunction(taskId); } @Override @@ -53,4 +55,10 @@ public SegmentIdWithShardSpec allocate( { return taskClient.allocateSegment(supervisorTaskId, row.getTimestamp()); } + + @Override + public SequenceNameFunction getSequenceNameFunction() + { + return sequenceNameFunction; + } } 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 c069c73cec70..20a7da6e8be1 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 @@ -31,7 +31,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; -import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialRangeSegmentGenerateTask; import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.query.Query; @@ -63,7 +62,6 @@ // for backward compatibility @Type(name = SinglePhaseSubTask.OLD_TYPE_NAME, value = LegacySinglePhaseSubTask.class), @Type(name = PartialHashSegmentGenerateTask.TYPE, value = PartialHashSegmentGenerateTask.class), - @Type(name = PartialHashSegmentMergeTask.TYPE, value = PartialHashSegmentMergeTask.class), @Type(name = PartialRangeSegmentGenerateTask.TYPE, value = PartialRangeSegmentGenerateTask.class), @Type(name = PartialDimensionDistributionTask.TYPE, value = PartialDimensionDistributionTask.class), @Type(name = PartialGenericSegmentMergeTask.TYPE, value = PartialGenericSegmentMergeTask.class), @@ -146,6 +144,11 @@ default int getPriority() */ QueryRunner getQueryRunner(Query query); + /** + * @return true if this Task type is queryable, such as streaming ingestion tasks + */ + boolean supportsQueries(); + /** * Returns an extra classpath that should be prepended to the default classpath when running this task. If no * extra classpath should be prepended, this should return null or the empty string. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java index bbfd1e2ddfa2..74c4c1738b0d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocation.java @@ -22,14 +22,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.BuildingShardSpec; import org.joda.time.Interval; /** * This class represents the intermediary data server where the partition of {@code interval} and {@code shardSpec} * is stored. */ -public class GenericPartitionLocation extends PartitionLocation +public class GenericPartitionLocation extends PartitionLocation { @JsonCreator public GenericPartitionLocation( @@ -38,7 +38,7 @@ public GenericPartitionLocation( @JsonProperty("useHttps") boolean useHttps, @JsonProperty("subTaskId") String subTaskId, @JsonProperty("interval") Interval interval, - @JsonProperty("shardSpec") ShardSpec shardSpec + @JsonProperty("shardSpec") BuildingShardSpec shardSpec ) { super(host, port, useHttps, subTaskId, interval, shardSpec); @@ -46,13 +46,13 @@ public GenericPartitionLocation( @JsonIgnore @Override - public int getPartitionId() + public int getBucketId() { - return getSecondaryPartition().getPartitionNum(); + return getSecondaryPartition().getBucketId(); } @JsonProperty - ShardSpec getShardSpec() + BuildingShardSpec getShardSpec() { return getSecondaryPartition(); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java index 5f4d16db2b19..a4ac80bdec04 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStat.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; @@ -33,12 +34,12 @@ * partition key). The {@link ShardSpec} is later used by {@link PartialGenericSegmentMergeTask} to merge the partial * segments. */ -public class GenericPartitionStat extends PartitionStat +public class GenericPartitionStat extends PartitionStat { private static final String PROP_SHARD_SPEC = "shardSpec"; // Secondary partition key - private final ShardSpec shardSpec; + private final BucketNumberedShardSpec shardSpec; @JsonCreator public GenericPartitionStat( @@ -46,7 +47,7 @@ public GenericPartitionStat( @JsonProperty("taskExecutorPort") int taskExecutorPort, @JsonProperty("useHttps") boolean useHttps, @JsonProperty("interval") Interval interval, - @JsonProperty(PROP_SHARD_SPEC) ShardSpec shardSpec, + @JsonProperty(PROP_SHARD_SPEC) BucketNumberedShardSpec shardSpec, @JsonProperty("numRows") @Nullable Integer numRows, @JsonProperty("sizeBytes") @Nullable Long sizeBytes ) @@ -56,14 +57,14 @@ public GenericPartitionStat( } @Override - public int getPartitionId() + public int getBucketId() { - return shardSpec.getPartitionNum(); + return shardSpec.getBucketId(); } @JsonProperty(PROP_SHARD_SPEC) @Override - ShardSpec getSecondaryPartition() + BucketNumberedShardSpec getSecondaryPartition() { return shardSpec; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java deleted file mode 100644 index 604eb7a6f504..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionLocation.java +++ /dev/null @@ -1,51 +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.indexing.common.task.batch.parallel; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -/** - * This class represents the intermediary data server where the partition of {@code interval} and {@code partitionId} - * is stored. - */ -public class HashPartitionLocation extends PartitionLocation -{ - @JsonCreator - public HashPartitionLocation( - @JsonProperty("host") String host, - @JsonProperty("port") int port, - @JsonProperty("useHttps") boolean useHttps, - @JsonProperty("subTaskId") String subTaskId, - @JsonProperty("interval") Interval interval, - @JsonProperty("partitionId") int partitionId - ) - { - super(host, port, useHttps, subTaskId, interval, partitionId); - } - - @JsonProperty - @Override - public int getPartitionId() - { - return getSecondaryPartition(); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java deleted file mode 100644 index 21019abe0fe7..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStat.java +++ /dev/null @@ -1,90 +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.indexing.common.task.batch.parallel; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Objects; - -/** - * Statistics about a partition created by {@link PartialHashSegmentGenerateTask}. Each partition is a set of data - * of the same time chunk (primary partition key) and the same partitionId (secondary partition key). This class - * holds the statistics of a single partition created by a task. - */ -public class HashPartitionStat extends PartitionStat -{ - // Secondary partition key - private final int partitionId; - - @JsonCreator - public HashPartitionStat( - @JsonProperty("taskExecutorHost") String taskExecutorHost, - @JsonProperty("taskExecutorPort") int taskExecutorPort, - @JsonProperty("useHttps") boolean useHttps, - @JsonProperty("interval") Interval interval, - @JsonProperty("partitionId") int partitionId, - @JsonProperty("numRows") @Nullable Integer numRows, - @JsonProperty("sizeBytes") @Nullable Long sizeBytes - ) - { - super(taskExecutorHost, taskExecutorPort, useHttps, interval, numRows, sizeBytes); - this.partitionId = partitionId; - } - - @JsonProperty - @Override - public int getPartitionId() - { - return partitionId; - } - - @JsonIgnore - @Override - Integer getSecondaryPartition() - { - return partitionId; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - HashPartitionStat that = (HashPartitionStat) o; - return partitionId == that.partitionId; - } - - @Override - public int hashCode() - { - return Objects.hash(super.hashCode(), partitionId); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java index 2d2ec3a0f2b6..bed85dea6926 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java @@ -29,6 +29,8 @@ import com.google.common.base.Throwables; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputFormat; @@ -78,6 +80,8 @@ import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.apache.druid.utils.CollectionUtils; @@ -335,24 +339,6 @@ PartialRangeSegmentGenerateParallelIndexTaskRunner createPartialRangeSegmentGene ); } - @VisibleForTesting - PartialHashSegmentMergeParallelIndexTaskRunner createPartialHashSegmentMergeRunner( - TaskToolbox toolbox, - List ioConfigs - ) - { - return new PartialHashSegmentMergeParallelIndexTaskRunner( - toolbox, - getId(), - getGroupId(), - getIngestionSchema().getDataSchema(), - ioConfigs, - getIngestionSchema().getTuningConfig(), - getContext(), - indexingServiceClient - ); - } - @VisibleForTesting PartialGenericSegmentMergeParallelIndexTaskRunner createPartialGenericSegmentMergeRunner( TaskToolbox toolbox, @@ -543,10 +529,8 @@ private TaskStatus runMultiPhaseParallel(TaskToolbox toolbox) throws Exception private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throws Exception { // 1. Partial segment generation phase - ParallelIndexTaskRunner indexingRunner = createRunner( - toolbox, - this::createPartialHashSegmentGenerateRunner - ); + ParallelIndexTaskRunner> indexingRunner + = createRunner(toolbox, this::createPartialHashSegmentGenerateRunner); TaskState state = runNextPhase(indexingRunner); if (state.isFailure()) { @@ -556,16 +540,16 @@ private TaskStatus runHashPartitionMultiPhaseParallel(TaskToolbox toolbox) throw // 2. Partial segment merge phase // partition (interval, partitionId) -> partition locations - Map, List> partitionToLocations = - groupHashPartitionLocationsPerPartition(indexingRunner.getReports()); - final List ioConfigs = createHashMergeIOConfigs( + Map, List> partitionToLocations = + groupGenericPartitionLocationsPerPartition(indexingRunner.getReports()); + final List ioConfigs = createGenericMergeIOConfigs( ingestionSchema.getTuningConfig().getTotalNumMergeTasks(), partitionToLocations ); - final ParallelIndexTaskRunner mergeRunner = createRunner( + final ParallelIndexTaskRunner mergeRunner = createRunner( toolbox, - tb -> createPartialHashSegmentMergeRunner(tb, ioConfigs) + tb -> createPartialGenericSegmentMergeRunner(tb, ioConfigs) ); state = runNextPhase(mergeRunner); if (state.isSuccess()) { @@ -658,38 +642,35 @@ private PartitionBoundaries determineRangePartition(Collection, List> groupHashPartitionLocationsPerPartition( - Map subTaskIdToReport - ) - { - BiFunction createPartitionLocationFunction = - (subtaskId, partitionStat) -> - new HashPartitionLocation( - partitionStat.getTaskExecutorHost(), - partitionStat.getTaskExecutorPort(), - partitionStat.isUseHttps(), - subtaskId, - partitionStat.getInterval(), - partitionStat.getSecondaryPartition() - ); - - return groupPartitionLocationsPerPartition(subTaskIdToReport, createPartitionLocationFunction); - } - private static Map, List> groupGenericPartitionLocationsPerPartition( Map> subTaskIdToReport ) { - BiFunction createPartitionLocationFunction = - (subtaskId, partitionStat) -> - new GenericPartitionLocation( - partitionStat.getTaskExecutorHost(), - partitionStat.getTaskExecutorPort(), - partitionStat.isUseHttps(), - subtaskId, - partitionStat.getInterval(), - partitionStat.getSecondaryPartition() - ); + final Map, BuildingShardSpec> intervalAndIntegerToShardSpec = new HashMap<>(); + final Object2IntMap intervalToNextPartitionId = new Object2IntOpenHashMap<>(); + final BiFunction createPartitionLocationFunction = + (subtaskId, partitionStat) -> { + final BuildingShardSpec shardSpec = intervalAndIntegerToShardSpec.computeIfAbsent( + Pair.of(partitionStat.getInterval(), partitionStat.getBucketId()), + key -> { + // Lazily determine the partitionId to create packed partitionIds for the core partitions. + // See the Javadoc of BucketNumberedShardSpec for details. + final int partitionId = intervalToNextPartitionId.computeInt( + partitionStat.getInterval(), + ((interval, nextPartitionId) -> nextPartitionId == null ? 0 : nextPartitionId + 1) + ); + return partitionStat.getSecondaryPartition().convert(partitionId); + } + ); + return new GenericPartitionLocation( + partitionStat.getTaskExecutorHost(), + partitionStat.getTaskExecutorPort(), + partitionStat.isUseHttps(), + subtaskId, + partitionStat.getInterval(), + shardSpec + ); + }; return groupPartitionLocationsPerPartition(subTaskIdToReport, createPartitionLocationFunction); } @@ -707,7 +688,7 @@ Map, List> groupPartitionLocationsPerPartition( final GeneratedPartitionsReport report = entry.getValue(); for (S partitionStat : report.getPartitionStats()) { final List locationsOfSamePartition = partitionToLocations.computeIfAbsent( - Pair.of(partitionStat.getInterval(), partitionStat.getPartitionId()), + Pair.of(partitionStat.getInterval(), partitionStat.getBucketId()), k -> new ArrayList<>() ); locationsOfSamePartition.add(createPartitionLocationFunction.apply(subTaskId, partitionStat)); @@ -717,18 +698,6 @@ Map, List> groupPartitionLocationsPerPartition( return partitionToLocations; } - private static List createHashMergeIOConfigs( - int totalNumMergeTasks, - Map, List> partitionToLocations - ) - { - return createMergeIOConfigs( - totalNumMergeTasks, - partitionToLocations, - PartialHashSegmentMergeIOConfig::new - ); - } - private static List createGenericMergeIOConfigs( int totalNumMergeTasks, Map, List> partitionToLocations @@ -984,7 +953,7 @@ SegmentIdWithShardSpec allocateNewSegment(DateTime timestamp) throws IOException dataSource, interval, version, - new NumberedShardSpec(partitionNum, 0) + new BuildingNumberedShardSpec(partitionNum) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java index 49b2b48ecd61..05103e85c6ab 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTaskRunner.java @@ -38,8 +38,9 @@ * uses {@link SinglePhaseParallelIndexTaskRunner} for it. * * For perfect rollup, parallel indexing is executed in multiple phases. The supervisor task currently uses - * {@link PartialHashSegmentGenerateParallelIndexTaskRunner} and {@link PartialHashSegmentMergeParallelIndexTaskRunner}, - * and can use more runners in the future. + * {@link PartialHashSegmentGenerateParallelIndexTaskRunner}, {@link PartialRangeSegmentGenerateParallelIndexTaskRunner}, + * and {@link PartialGenericSegmentMergeParallelIndexTaskRunner}. + * More runners can be added in the future. */ public interface ParallelIndexTaskRunner { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java index 73e2ac7c7414..935eeb48c21e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexTuningConfig.java @@ -57,14 +57,14 @@ public class ParallelIndexTuningConfig extends IndexTuningConfig /** * Max number of segments to merge at the same time. - * Used only by {@link PartialHashSegmentMergeTask}. + * Used only by {@link PartialGenericSegmentMergeTask}. * This configuration was temporarily added to avoid using too much memory while merging segments, * and will be removed once {@link org.apache.druid.segment.IndexMerger} is improved to not use much memory. */ private final int maxNumSegmentsToMerge; /** - * Total number of tasks for partial segment merge (that is, number of {@link PartialHashSegmentMergeTask}s). + * Total number of tasks for partial segment merge (that is, number of {@link PartialGenericSegmentMergeTask}s). * Used only when this task runs with shuffle. */ private final int totalNumMergeTasks; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java index 858eff4ae63d..fed80d9252f8 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialGenericSegmentMergeTask.java @@ -29,6 +29,8 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.TaskResource; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.timeline.partition.BuildingShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.joda.time.Interval; @@ -39,12 +41,12 @@ /** * {@link ParallelIndexTaskRunner} for the phase to merge generic partitioned segments in multi-phase parallel indexing. */ -public class PartialGenericSegmentMergeTask extends PartialSegmentMergeTask +public class PartialGenericSegmentMergeTask extends PartialSegmentMergeTask { public static final String TYPE = "partial_index_generic_merge"; private final PartialGenericSegmentMergeIngestionSpec ingestionSchema; - private final Table intervalAndIntegerToShardSpec; + private final Table> intervalAndIntegerToShardSpec; @JsonCreator public PartialGenericSegmentMergeTask( @@ -82,24 +84,28 @@ public PartialGenericSegmentMergeTask( ); } - private static Table createIntervalAndIntegerToShardSpec( + private static Table> createIntervalAndIntegerToShardSpec( List partitionLocations ) { - Table intervalAndIntegerToShardSpec = HashBasedTable.create(); + final Table> intervalAndIntegerToShardSpec = HashBasedTable.create(); partitionLocations.forEach( p -> { - ShardSpec currShardSpec = intervalAndIntegerToShardSpec.get(p.getInterval(), p.getPartitionId()); - Preconditions.checkArgument( - currShardSpec == null || p.getShardSpec().equals(currShardSpec), - "interval %s, partitionId %s mismatched shard specs: %s", - p.getInterval(), - p.getPartitionId(), - partitionLocations - ); - - intervalAndIntegerToShardSpec.put(p.getInterval(), p.getPartitionId(), p.getShardSpec()); + final ShardSpec currShardSpec = intervalAndIntegerToShardSpec.get(p.getInterval(), p.getBucketId()); + if (currShardSpec == null) { + intervalAndIntegerToShardSpec.put(p.getInterval(), p.getBucketId(), p.getShardSpec()); + } else { + if (!p.getShardSpec().equals(currShardSpec)) { + throw new ISE( + "interval %s, bucketId %s mismatched shard specs: %s and %s", + p.getInterval(), + p.getBucketId(), + currShardSpec, + p.getShardSpec() + ); + } + } } ); @@ -119,7 +125,7 @@ public String getType() } @Override - ShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) + BuildingShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) { return Preconditions.checkNotNull( intervalAndIntegerToShardSpec.get(interval, partitionId), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java index ef8869c42727..e067eb909196 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateParallelIndexTaskRunner.java @@ -27,11 +27,9 @@ /** * {@link ParallelIndexTaskRunner} for the phase to create hash partitioned segments in multi-phase parallel indexing. - * - * @see PartialHashSegmentMergeParallelIndexTaskRunner */ class PartialHashSegmentGenerateParallelIndexTaskRunner - extends InputSourceSplitParallelIndexTaskRunner + extends InputSourceSplitParallelIndexTaskRunner> { private static final String PHASE_NAME = "partial segment generation"; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java index 7be462f7840b..1bfda30522af 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentGenerateTask.java @@ -26,8 +26,8 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder; @@ -35,6 +35,7 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartialShardSpec; import org.joda.time.Interval; @@ -51,7 +52,7 @@ * hashing the segment granularity and partition dimensions in {@link HashedPartitionsSpec}. Partitioned segments are * stored in local storage using {@link org.apache.druid.indexing.worker.ShuffleDataSegmentPusher}. */ -public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask +public class PartialHashSegmentGenerateTask extends PartialSegmentGenerateTask { public static final String TYPE = "partial_index_generate"; private static final String PROP_SPEC = "spec"; @@ -127,7 +128,7 @@ public boolean isReady(TaskActionClient taskActionClient) throws Exception } @Override - CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) + SegmentAllocatorForBatch createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) throws IOException { final GranularitySpec granularitySpec = ingestionSchema.getDataSchema().getGranularitySpec(); @@ -137,28 +138,29 @@ CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelInde toolbox, getDataSource(), getId(), + granularitySpec, new SupervisorTaskAccess(supervisorTaskId, taskClient), createHashPartitionAnalysisFromPartitionsSpec(granularitySpec, partitionsSpec) ); } @Override - GeneratedHashPartitionsReport createGeneratedPartitionsReport(TaskToolbox toolbox, List segments) + GeneratedPartitionsMetadataReport createGeneratedPartitionsReport(TaskToolbox toolbox, List segments) { - List partitionStats = segments.stream() - .map(segment -> createPartitionStat(toolbox, segment)) - .collect(Collectors.toList()); - return new GeneratedHashPartitionsReport(getId(), partitionStats); + List partitionStats = segments.stream() + .map(segment -> createPartitionStat(toolbox, segment)) + .collect(Collectors.toList()); + return new GeneratedPartitionsMetadataReport(getId(), partitionStats); } - private HashPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment) + private GenericPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegment segment) { - return new HashPartitionStat( + return new GenericPartitionStat( toolbox.getTaskExecutorNode().getHost(), toolbox.getTaskExecutorNode().getPortToUse(), toolbox.getTaskExecutorNode().isEnableTlsPort(), segment.getInterval(), - segment.getShardSpec().getPartitionNum(), + (BucketNumberedShardSpec) segment.getShardSpec(), null, // numRows is not supported yet null // sizeBytes is not supported yet ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java deleted file mode 100644 index c693513c6527..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeParallelIndexTaskRunner.java +++ /dev/null @@ -1,115 +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.indexing.common.task.batch.parallel; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.data.input.InputSplit; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.segment.indexing.DataSchema; - -import java.util.Iterator; -import java.util.List; -import java.util.Map; - -/** - * {@link ParallelIndexTaskRunner} for the phase to merge hash partitioned segments in multi-phase parallel indexing. - * - * @see PartialHashSegmentGenerateParallelIndexTaskRunner - */ -class PartialHashSegmentMergeParallelIndexTaskRunner - extends ParallelIndexPhaseRunner -{ - private static final String PHASE_NAME = "partial segment merge"; - - private final DataSchema dataSchema; - private final List mergeIOConfigs; - - PartialHashSegmentMergeParallelIndexTaskRunner( - TaskToolbox toolbox, - String taskId, - String groupId, - DataSchema dataSchema, - List mergeIOConfigs, - ParallelIndexTuningConfig tuningConfig, - Map context, - IndexingServiceClient indexingServiceClient - ) - { - super(toolbox, taskId, groupId, tuningConfig, context, indexingServiceClient); - - this.dataSchema = dataSchema; - this.mergeIOConfigs = mergeIOConfigs; - } - - @Override - public String getName() - { - return PHASE_NAME; - } - - @Override - Iterator> subTaskSpecIterator() - { - return mergeIOConfigs.stream().map(this::newTaskSpec).iterator(); - } - - @Override - int estimateTotalNumSubTasks() - { - return mergeIOConfigs.size(); - } - - @VisibleForTesting - SubTaskSpec newTaskSpec(PartialHashSegmentMergeIOConfig ioConfig) - { - final PartialHashSegmentMergeIngestionSpec ingestionSpec = - new PartialHashSegmentMergeIngestionSpec( - dataSchema, - ioConfig, - getTuningConfig() - ); - return new SubTaskSpec( - getTaskId() + "_" + getAndIncrementNextSpecId(), - getGroupId(), - getTaskId(), - getContext(), - new InputSplit<>(ioConfig.getPartitionLocations()) - ) - { - @Override - public PartialHashSegmentMergeTask newSubTask(int numAttempts) - { - return new PartialHashSegmentMergeTask( - null, - getGroupId(), - null, - getSupervisorTaskId(), - numAttempts, - ingestionSpec, - getContext(), - null, - null, - null - ); - } - }; - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java deleted file mode 100644 index cf5aaea2edc9..000000000000 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTask.java +++ /dev/null @@ -1,112 +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.indexing.common.task.batch.parallel; - -import com.fasterxml.jackson.annotation.JacksonInject; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.google.common.base.Preconditions; -import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.indexer.partitions.PartitionsSpec; -import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.task.IndexTaskClientFactory; -import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Map; - -/** - * The worker task of {@link PartialHashSegmentMergeParallelIndexTaskRunner}. This task reads partitioned segments - * created by {@link PartialHashSegmentGenerateTask}s, merges them, and pushes to deep storage. The pushed segments are - * reported to {@link PartialHashSegmentMergeParallelIndexTaskRunner}. - */ - -public class PartialHashSegmentMergeTask - extends PartialSegmentMergeTask -{ - public static final String TYPE = "partial_index_merge"; - - private final HashedPartitionsSpec partitionsSpec; - private final PartialHashSegmentMergeIngestionSpec ingestionSchema; - - @JsonCreator - public PartialHashSegmentMergeTask( - // id shouldn't be null except when this task is created by ParallelIndexSupervisorTask - @JsonProperty("id") @Nullable String id, - @JsonProperty("groupId") final String groupId, - @JsonProperty("resource") final TaskResource taskResource, - @JsonProperty("supervisorTaskId") final String supervisorTaskId, - @JsonProperty("numAttempts") final int numAttempts, // zero-based counting - @JsonProperty("spec") final PartialHashSegmentMergeIngestionSpec ingestionSchema, - @JsonProperty("context") final Map context, - @JacksonInject IndexingServiceClient indexingServiceClient, - @JacksonInject IndexTaskClientFactory taskClientFactory, - @JacksonInject ShuffleClient shuffleClient - ) - { - super( - getOrMakeId(id, TYPE, ingestionSchema.getDataSchema().getDataSource()), - groupId, - taskResource, - supervisorTaskId, - ingestionSchema.getDataSchema(), - ingestionSchema.getIOConfig(), - ingestionSchema.getTuningConfig(), - numAttempts, - context, - indexingServiceClient, - taskClientFactory, - shuffleClient - ); - - this.ingestionSchema = ingestionSchema; - - PartitionsSpec inputPartitionsSpec = ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec(); - Preconditions.checkArgument(inputPartitionsSpec instanceof HashedPartitionsSpec, "hashed partitionsSpec required"); - partitionsSpec = (HashedPartitionsSpec) inputPartitionsSpec; - Preconditions.checkNotNull(partitionsSpec.getNumShards(), "hashed partitionsSpec numShards required"); - } - - @JsonProperty("spec") - private PartialHashSegmentMergeIngestionSpec getIngestionSchema() - { - return ingestionSchema; - } - - @Override - public String getType() - { - return TYPE; - } - - @Override - HashBasedNumberedShardSpec createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId) - { - return new HashBasedNumberedShardSpec( - partitionId, - Preconditions.checkNotNull(partitionsSpec.getNumShards(), "numShards"), - partitionsSpec.getPartitionDimensions(), - toolbox.getJsonMapper() - ); - } -} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java index e0f9461d166b..39a7e6576c90 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateParallelIndexTaskRunner.java @@ -29,8 +29,6 @@ /** * {@link ParallelIndexTaskRunner} for the phase to create range partitioned segments in multi-phase parallel indexing. - * - * @see PartialHashSegmentMergeParallelIndexTaskRunner */ class PartialRangeSegmentGenerateParallelIndexTaskRunner extends InputSourceSplitParallelIndexTaskRunner> diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java index 4236226f3eb6..949c3748ac4a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialRangeSegmentGenerateTask.java @@ -28,8 +28,8 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; +import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.batch.parallel.iterator.RangePartitionIndexTaskInputRowIteratorBuilder; @@ -37,6 +37,7 @@ import org.apache.druid.indexing.worker.ShuffleDataSegmentPusher; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; import org.joda.time.Interval; @@ -150,7 +151,7 @@ public boolean isReady(TaskActionClient taskActionClient) } @Override - CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) + SegmentAllocatorForBatch createSegmentAllocator(TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient) throws IOException { final RangePartitionAnalysis partitionAnalysis = new RangePartitionAnalysis( @@ -161,6 +162,7 @@ CachingSegmentAllocator createSegmentAllocator(TaskToolbox toolbox, ParallelInde toolbox, getDataSource(), getId(), + ingestionSchema.getDataSchema().getGranularitySpec(), new SupervisorTaskAccess(supervisorTaskId, taskClient), partitionAnalysis ); @@ -182,7 +184,7 @@ private GenericPartitionStat createPartitionStat(TaskToolbox toolbox, DataSegmen toolbox.getTaskExecutorNode().getPortToUse(), toolbox.getTaskExecutorNode().isEnableTlsPort(), segment.getInterval(), - segment.getShardSpec(), + (BucketNumberedShardSpec) segment.getShardSpec(), null, // numRows is not supported yet null // sizeBytes is not supported yet ); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java index 65e570a9d84d..cbde9285cded 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java @@ -28,11 +28,10 @@ import org.apache.druid.indexing.common.stats.DropwizardRowIngestionMeters; import org.apache.druid.indexing.common.stats.RowIngestionMeters; import org.apache.druid.indexing.common.task.BatchAppenderators; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider; import org.apache.druid.indexing.common.task.IndexTaskClientFactory; import org.apache.druid.indexing.common.task.InputSourceProcessor; -import org.apache.druid.indexing.common.task.NonLinearlyPartitionedSequenceNameFunction; +import org.apache.druid.indexing.common.task.SegmentAllocatorForBatch; import org.apache.druid.indexing.common.task.SequenceNameFunction; import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; @@ -129,7 +128,7 @@ public final TaskStatus runTask(TaskToolbox toolbox) throws Exception /** * @return {@link SegmentAllocator} suitable for the desired segment partitioning strategy. */ - abstract CachingSegmentAllocator createSegmentAllocator( + abstract SegmentAllocatorForBatch createSegmentAllocator( TaskToolbox toolbox, ParallelIndexSupervisorTaskClient taskClient ) throws IOException; @@ -171,11 +170,8 @@ private List generateSegments( final PartitionsSpec partitionsSpec = tuningConfig.getGivenOrDefaultPartitionsSpec(); final long pushTimeout = tuningConfig.getPushTimeout(); - final CachingSegmentAllocator segmentAllocator = createSegmentAllocator(toolbox, taskClient); - final SequenceNameFunction sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction( - getId(), - segmentAllocator.getShardSpecs() - ); + final SegmentAllocatorForBatch segmentAllocator = createSegmentAllocator(toolbox, taskClient); + final SequenceNameFunction sequenceNameFunction = segmentAllocator.getSequenceNameFunction(); final Appenderator appenderator = BatchAppenderators.newAppenderator( getId(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java index 96596be4dc73..b105d5e8da09 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentMergeTask.java @@ -139,10 +139,10 @@ public boolean isReady(TaskActionClient taskActionClient) public TaskStatus runTask(TaskToolbox toolbox) throws Exception { // Group partitionLocations by interval and partitionId - final Map>> intervalToPartitions = new HashMap<>(); + final Map>> intervalToBuckets = new HashMap<>(); for (P location : ioConfig.getPartitionLocations()) { - intervalToPartitions.computeIfAbsent(location.getInterval(), k -> new Int2ObjectOpenHashMap<>()) - .computeIfAbsent(location.getPartitionId(), k -> new ArrayList<>()) + intervalToBuckets.computeIfAbsent(location.getInterval(), k -> new Int2ObjectOpenHashMap<>()) + .computeIfAbsent(location.getBucketId(), k -> new ArrayList<>()) .add(location); } @@ -168,7 +168,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception final Stopwatch fetchStopwatch = Stopwatch.createStarted(); final Map>> intervalToUnzippedFiles = fetchSegmentFiles( toolbox, - intervalToPartitions + intervalToBuckets ); final long fetchTime = fetchStopwatch.elapsed(TimeUnit.SECONDS); fetchStopwatch.stop(); @@ -202,7 +202,7 @@ public TaskStatus runTask(TaskToolbox toolbox) throws Exception private Map>> fetchSegmentFiles( TaskToolbox toolbox, - Map>> intervalToPartitions + Map>> intervalToBuckets ) throws IOException { final File tempDir = toolbox.getIndexingTmpDir(); @@ -211,26 +211,26 @@ private Map>> fetchSegmentFiles( final Map>> intervalToUnzippedFiles = new HashMap<>(); // Fetch partition files - for (Entry>> entryPerInterval : intervalToPartitions.entrySet()) { + for (Entry>> entryPerInterval : intervalToBuckets.entrySet()) { final Interval interval = entryPerInterval.getKey(); - for (Int2ObjectMap.Entry> entryPerPartitionId : + for (Int2ObjectMap.Entry> entryPerBucketId : entryPerInterval.getValue().int2ObjectEntrySet()) { - final int partitionId = entryPerPartitionId.getIntKey(); + final int bucketId = entryPerBucketId.getIntKey(); final File partitionDir = FileUtils.getFile( tempDir, interval.getStart().toString(), interval.getEnd().toString(), - Integer.toString(partitionId) + Integer.toString(bucketId) ); FileUtils.forceMkdir(partitionDir); - for (P location : entryPerPartitionId.getValue()) { + for (P location : entryPerBucketId.getValue()) { final File zippedFile = shuffleClient.fetchSegmentFile(partitionDir, supervisorTaskId, location); try { final File unzippedDir = new File(partitionDir, StringUtils.format("unzipped_%s", location.getSubTaskId())); FileUtils.forceMkdir(unzippedDir); CompressionUtils.unzip(zippedFile, unzippedDir); intervalToUnzippedFiles.computeIfAbsent(interval, k -> new Int2ObjectOpenHashMap<>()) - .computeIfAbsent(partitionId, k -> new ArrayList<>()) + .computeIfAbsent(bucketId, k -> new ArrayList<>()) .add(unzippedDir); } finally { @@ -247,7 +247,7 @@ private Map>> fetchSegmentFiles( /** * Create a {@link ShardSpec} suitable for the desired secondary partitioning strategy. */ - abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int partitionId); + abstract S createShardSpec(TaskToolbox toolbox, Interval interval, int bucketId); private Set mergeAndPushSegments( TaskToolbox toolbox, @@ -262,9 +262,9 @@ private Set mergeAndPushSegments( final Set pushedSegments = new HashSet<>(); for (Entry>> entryPerInterval : intervalToUnzippedFiles.entrySet()) { final Interval interval = entryPerInterval.getKey(); - for (Int2ObjectMap.Entry> entryPerPartitionId : entryPerInterval.getValue().int2ObjectEntrySet()) { - final int partitionId = entryPerPartitionId.getIntKey(); - final List segmentFilesToMerge = entryPerPartitionId.getValue(); + for (Int2ObjectMap.Entry> entryPerBucketId : entryPerInterval.getValue().int2ObjectEntrySet()) { + final int bucketId = entryPerBucketId.getIntKey(); + final List segmentFilesToMerge = entryPerBucketId.getValue(); final Pair> mergedFileAndDimensionNames = mergeSegmentsInSamePartition( dataSchema, tuningConfig, @@ -290,7 +290,7 @@ private Set mergeAndPushSegments( null, // will be filled in the segmentPusher mergedFileAndDimensionNames.rhs, metricNames, - createShardSpec(toolbox, interval, partitionId), + createShardSpec(toolbox, interval, bucketId), null, // will be filled in the segmentPusher 0 // will be filled in the segmentPusher ), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java index e6578c50924f..da382cec0604 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionLocation.java @@ -29,7 +29,7 @@ /** * This class represents the intermediary data server where the partition of {@link #interval} and - * {@link #getPartitionId()} is stored. + * {@link #getBucketId()} is stored. */ abstract class PartitionLocation { @@ -93,13 +93,13 @@ public T getSecondaryPartition() return secondaryPartition; } - abstract int getPartitionId(); + abstract int getBucketId(); final URI toIntermediaryDataServerURI(String supervisorTaskId) { return URI.create( StringUtils.format( - "%s://%s:%d/druid/worker/v1/shuffle/task/%s/%s/partition?startTime=%s&endTime=%s&partitionId=%d", + "%s://%s:%d/druid/worker/v1/shuffle/task/%s/%s/partition?startTime=%s&endTime=%s&bucketId=%d", useHttps ? "https" : "http", host, port, @@ -107,7 +107,7 @@ final URI toIntermediaryDataServerURI(String supervisorTaskId) StringUtils.urlEncode(subTaskId), interval.getStart(), interval.getEnd(), - getPartitionId() + getBucketId() ) ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java index 66974c297eea..c7f1a55c2a2e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartitionStat.java @@ -104,7 +104,7 @@ public final Long getSizeBytes() /** * @return Uniquely identifying index from 0..N-1 of the N partitions */ - abstract int getPartitionId(); + abstract int getBucketId(); /** * @return Definition of secondary partition. For example, for range partitioning, this should include the start/end. diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java index 9f1dc52c1ff4..61908a8f255b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java @@ -316,6 +316,7 @@ private Set generateAndPushSegments( final boolean explicitIntervals = granularitySpec.bucketIntervals().isPresent(); final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning( toolbox, + getId(), new SupervisorTaskAccess(getSupervisorTaskId(), taskClient), getIngestionSchema().getDataSchema(), getTaskLockHelper(), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java index 564b3af8ab6f..26f20f6f8688 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SubTaskReport.java @@ -30,7 +30,6 @@ @JsonTypeInfo(use = Id.NAME, property = "type", defaultImpl = PushedSegmentsReport.class) @JsonSubTypes(value = { @Type(name = PushedSegmentsReport.TYPE, value = PushedSegmentsReport.class), - @Type(name = GeneratedHashPartitionsReport.TYPE, value = GeneratedHashPartitionsReport.class), @Type(name = DimensionDistributionReport.TYPE, value = DimensionDistributionReport.class), @Type(name = GeneratedPartitionsMetadataReport.TYPE, value = GeneratedPartitionsMetadataReport.class) }) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java index 2e4328043f43..efecdda5e096 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/CompletePartitionAnalysis.java @@ -21,12 +21,11 @@ import org.apache.druid.indexer.partitions.PartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.joda.time.Interval; import java.util.List; import java.util.Map; -import java.util.function.Function; /** * This interface represents the PartitionAnalysis that has the complete picture of secondary partitions to create. @@ -35,9 +34,5 @@ */ public interface CompletePartitionAnalysis extends PartitionAnalysis { - Map> convertToIntervalToSegmentIds( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ); + Map>> createBuckets(TaskToolbox toolbox); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java index a4b3a86a1b55..5773f095c10f 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/HashPartitionAnalysis.java @@ -23,8 +23,8 @@ import org.apache.druid.indexer.partitions.HashedPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; import org.joda.time.Interval; import java.util.Collections; @@ -33,7 +33,6 @@ import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -92,37 +91,23 @@ public void forEach(BiConsumer consumer) } @Override - public Map> convertToIntervalToSegmentIds( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ) + public Map>> createBuckets(TaskToolbox toolbox) { - final Map> intervalToSegmentIds = - Maps.newHashMapWithExpectedSize(getNumTimePartitions()); - + final Map>> intervalToLookup = Maps.newHashMapWithExpectedSize( + intervalToNumBuckets.size() + ); forEach((interval, numBuckets) -> { - intervalToSegmentIds.put( - interval, - IntStream.range(0, numBuckets) - .mapToObj(i -> { - final HashBasedNumberedShardSpec shardSpec = new HashBasedNumberedShardSpec( - i, - numBuckets, - partitionsSpec.getPartitionDimensions(), - toolbox.getJsonMapper() - ); - return new SegmentIdWithShardSpec( - dataSource, - interval, - versionFinder.apply(interval), - shardSpec - ); - }) - .collect(Collectors.toList()) - ); + final List> buckets = IntStream + .range(0, numBuckets) + .mapToObj(i -> new HashBucketShardSpec( + i, + numBuckets, + partitionsSpec.getPartitionDimensions(), + toolbox.getJsonMapper() + )) + .collect(Collectors.toList()); + intervalToLookup.put(interval, buckets); }); - - return intervalToSegmentIds; + return intervalToLookup; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java index b2753931ba1b..c8a2b8804d07 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/partition/RangePartitionAnalysis.java @@ -23,19 +23,17 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; import org.apache.druid.timeline.partition.PartitionBoundaries; -import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.apache.druid.timeline.partition.RangeBucketShardSpec; import org.joda.time.Interval; -import javax.annotation.Nullable; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.function.BiConsumer; -import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -90,44 +88,13 @@ public int getNumTimePartitions() return intervalToPartitionBoundaries.size(); } - @Override - public Map> convertToIntervalToSegmentIds( - TaskToolbox toolbox, - String dataSource, - Function versionFinder - ) - { - final String partitionDimension = partitionsSpec.getPartitionDimension(); - final Map> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( - getNumTimePartitions() - ); - - forEach((interval, partitionBoundaries) -> - intervalToSegmentIds.put( - interval, - translatePartitionBoundaries( - dataSource, - interval, - partitionDimension, - partitionBoundaries, - versionFinder - ) - ) - ); - - return intervalToSegmentIds; - } - /** * Translate {@link PartitionBoundaries} into the corresponding * {@link SingleDimensionPartitionsSpec} with segment id. */ - private static List translatePartitionBoundaries( - String dataSource, - Interval interval, + private static List> translatePartitionBoundaries( String partitionDimension, - PartitionBoundaries partitionBoundaries, - Function versionFinder + PartitionBoundaries partitionBoundaries ) { if (partitionBoundaries.isEmpty()) { @@ -135,40 +102,30 @@ private static List translatePartitionBoundaries( } return IntStream.range(0, partitionBoundaries.size() - 1) - .mapToObj(i -> createSegmentIdWithShardSpec( - dataSource, - interval, - versionFinder.apply(interval), + .mapToObj(i -> new RangeBucketShardSpec( + i, partitionDimension, partitionBoundaries.get(i), - partitionBoundaries.get(i + 1), - i + partitionBoundaries.get(i + 1) )) .collect(Collectors.toList()); } - private static SegmentIdWithShardSpec createSegmentIdWithShardSpec( - String dataSource, - Interval interval, - String version, - String partitionDimension, - String partitionStart, - @Nullable String partitionEnd, - int partitionNum - ) + @Override + public Map>> createBuckets(TaskToolbox toolbox) { - // The shardSpec created here will be reused in PartialGenericSegmentMergeTask. This is ok because - // all PartialSegmentGenerateTasks create the same set of segmentIds (and thus shardSpecs). - return new SegmentIdWithShardSpec( - dataSource, - interval, - version, - new SingleDimensionShardSpec( - partitionDimension, - partitionStart, - partitionEnd, - partitionNum - ) + final String partitionDimension = partitionsSpec.getPartitionDimension(); + final Map>> intervalToSegmentIds = Maps.newHashMapWithExpectedSize( + getNumTimePartitions() ); + + forEach((interval, partitionBoundaries) -> + intervalToSegmentIds.put( + interval, + translatePartitionBoundaries(partitionDimension, partitionBoundaries) + ) + ); + + return intervalToSegmentIds; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/GeneratorInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/GeneratorInputSource.java new file mode 100644 index 000000000000..208666d15b93 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/GeneratorInputSource.java @@ -0,0 +1,292 @@ +/* + * 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.indexing.input; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowListPlusRawValues; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSource; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.java.util.common.CloseableIterators; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorColumnSchema; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.Random; +import java.util.stream.LongStream; +import java.util.stream.Stream; + +/** + * {@link InputSource} that can be used to seed a Druid cluster with test data, using either the built-in schemas + * defined in {@link GeneratorBasicSchemas}, or by directly supplying a list of {@link GeneratorColumnSchema}, to + * construct a {@link DataGenerator}. To produce a stable set of data, a random {@link #seed} may be supplied which + * will be used for all data generated by the columns. When {@link #numSplits} is greater than 1, the {@link #seed} + * will be instead used to pick a new seed for each split, allowing the splits to produce a different set of data, + * but still in a stable manner. + */ +public class GeneratorInputSource extends AbstractInputSource implements SplittableInputSource +{ + private static final int DEFAULT_NUM_ROWS = 1000; + private static final int DEFAULT_NUM_SPLITS = 1; + private static final long DEFAULT_SEED = 1024L; + private static final long DEFAULT_START_TIME = DateTimes.nowUtc().minusDays(1).getMillis(); + private static final int DEFAULT_CONSECUTIVE_TIMESTAMPS = 100; + private static final double DEFAULT_TIMESTAMP_INCREMENT = 1.0; + + private final String schemaName; + private final List schema; + private final int numRows; + private final Integer numSplits; + private final Long seed; + private final Long startTime; + private final Integer numConsecutiveTimestamps; + private final Double timestampIncrement; + + @JsonCreator + public GeneratorInputSource( + @JsonProperty("schemaName") @Nullable String schemaName, + @JsonProperty("schema") @Nullable List schema, + @JsonProperty("numRows") Integer numRows, + @JsonProperty("numSplits") Integer numSplits, + @JsonProperty("seed") Long seed, + @JsonProperty("startTime") Long startTime, + @JsonProperty("numConsecutiveTimestamps") Integer numConsecutiveTimestamps, + @JsonProperty("timestampIncrement") Double timestampIncrement + ) + { + Preconditions.checkArgument( + schemaName != null || schema != null, + "Must specify either 'schemaName' or 'schema'" + ); + this.schemaName = schemaName; + this.schema = schema != null + ? schema + : GeneratorBasicSchemas.SCHEMA_MAP.get(schemaName).getColumnSchemas(); + this.numRows = numRows != null ? numRows : DEFAULT_NUM_ROWS; + this.numSplits = numSplits != null ? numSplits : DEFAULT_NUM_SPLITS; + this.seed = seed != null ? seed : DEFAULT_SEED; + this.startTime = startTime != null ? startTime : DEFAULT_START_TIME; + this.numConsecutiveTimestamps = numConsecutiveTimestamps != null + ? numConsecutiveTimestamps + : DEFAULT_CONSECUTIVE_TIMESTAMPS; + this.timestampIncrement = timestampIncrement != null ? timestampIncrement : DEFAULT_TIMESTAMP_INCREMENT; + } + + @Override + public Stream> createSplits( + InputFormat inputFormat, + @Nullable SplitHintSpec splitHintSpec + ) + { + Random r = new Random(seed); + return LongStream.range(0, numSplits).mapToObj(i -> new InputSplit<>(r.nextLong())); + } + + @Override + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return numSplits; + } + + @Override + public InputSource withSplit(InputSplit split) + { + return new GeneratorInputSource( + schemaName, + schema, + numRows, + 1, + split.get(), + startTime, + numConsecutiveTimestamps, + timestampIncrement + ); + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + { + return new InputSourceReader() + { + @Override + public CloseableIterator read() + { + return CloseableIterators.withEmptyBaggage(new Iterator() + { + int rowCount = 0; + private final DataGenerator generator = makeGenerator(); + + @Override + public boolean hasNext() + { + return rowCount < numRows; + } + + @Override + public InputRow next() + { + rowCount++; + return generator.nextRow(); + } + }); + } + + @Override + public CloseableIterator sample() + { + return CloseableIterators.withEmptyBaggage(new Iterator() + { + int rowCount = 0; + private final DataGenerator generator = makeGenerator(); + + @Override + public boolean hasNext() + { + return rowCount < numRows; + } + + @Override + public InputRowListPlusRawValues next() + { + rowCount++; + InputRow row = generator.nextRow(); + return InputRowListPlusRawValues.of(row, ((MapBasedInputRow) row).getEvent()); + } + }); + } + }; + } + + @JsonProperty + public String getSchemaName() + { + return schemaName; + } + + @JsonProperty + public List getSchema() + { + return schemaName == null ? schema : null; + } + + @JsonProperty + public int getNumRows() + { + return numRows; + } + + @JsonProperty + public Integer getNumSplits() + { + return numSplits; + } + + @JsonProperty + public Long getSeed() + { + return seed; + } + + @JsonProperty + public Long getStartTime() + { + return startTime; + } + + @JsonProperty + public Integer getNumConsecutiveTimestamps() + { + return numConsecutiveTimestamps; + } + + @JsonProperty + public Double getTimestampIncrement() + { + return timestampIncrement; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + GeneratorInputSource that = (GeneratorInputSource) o; + return numRows == that.numRows && + Objects.equals(schemaName, that.schemaName) && + Objects.equals(schema, that.schema) && + Objects.equals(numSplits, that.numSplits) && + Objects.equals(seed, that.seed) && + Objects.equals(startTime, that.startTime) && + Objects.equals(numConsecutiveTimestamps, that.numConsecutiveTimestamps) && + Objects.equals(timestampIncrement, that.timestampIncrement); + } + + @Override + public int hashCode() + { + return Objects.hash( + schemaName, + schema, + numRows, + numSplits, + seed, + startTime, + numConsecutiveTimestamps, + timestampIncrement + ); + } + + private DataGenerator makeGenerator() + { + return new DataGenerator( + schema, + seed, + startTime, + numConsecutiveTimestamps, + timestampIncrement + ); + } +} 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 2f1abc1d7f53..10f5e6b5e9c2 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 @@ -327,6 +327,13 @@ public TaskStatus call() command.add(nodeType); } + // If the task type is queryable, we need to load broadcast segments on the peon, used for + // join queries + if (task.supportsQueries()) { + command.add("--loadBroadcastSegments"); + command.add("true"); + } + if (!taskFile.exists()) { jsonMapper.writeValue(taskFile, task); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java index 16ad0864106f..6c7dd4e3bd5c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java @@ -19,12 +19,9 @@ package org.apache.druid.indexing.overlord; -import com.google.common.base.Function; import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; @@ -284,30 +281,19 @@ private void manage() throws InterruptedException } } // Kill tasks that shouldn't be running - final Set tasksToKill = Sets.difference( - runnerTaskFutures.keySet(), - ImmutableSet.copyOf( - Lists.transform( - tasks, - new Function() - { - @Override - public String apply(Task task) - { - return task.getId(); - } - } - ) - ) - ); + final Set knownTaskIds = tasks + .stream() + .map(Task::getId) + .collect(Collectors.toSet()); + final Set tasksToKill = Sets.difference(runnerTaskFutures.keySet(), knownTaskIds); if (!tasksToKill.isEmpty()) { log.info("Asking taskRunner to clean up %,d tasks.", tasksToKill.size()); for (final String taskId : tasksToKill) { try { taskRunner.shutdown( taskId, - "task is not in runnerTaskFutures[%s]", - runnerTaskFutures.keySet() + "task is not in knownTaskIds[%s]", + knownTaskIds ); } catch (Exception e) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerCategorySpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerCategorySpec.java index 57340bc8f48b..9f58d62d05a9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerCategorySpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/setup/WorkerCategorySpec.java @@ -38,7 +38,7 @@ public WorkerCategorySpec( @JsonProperty("strong") boolean strong ) { - this.categoryMap = categoryMap == null ? Collections.EMPTY_MAP : categoryMap; + this.categoryMap = categoryMap == null ? Collections.emptyMap() : categoryMap; this.strong = strong; } @@ -96,7 +96,7 @@ public CategoryConfig( ) { this.defaultCategory = defaultCategory; - this.categoryAffinity = categoryAffinity == null ? Collections.EMPTY_MAP : categoryAffinity; + this.categoryAffinity = categoryAffinity == null ? Collections.emptyMap() : categoryAffinity; } @JsonProperty diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java index 78090ca5181f..6df598c24031 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/IntermediaryDataManager.java @@ -70,7 +70,7 @@ * and phase 2 tasks read those files via HTTP. * * The directory where segment files are placed is structured as - * {@link StorageLocation#path}/supervisorTaskId/startTimeOfSegment/endTimeOfSegment/partitionIdOfSegment. + * {@link StorageLocation#path}/supervisorTaskId/startTimeOfSegment/endTimeOfSegment/bucketIdOfSegment. * * This class provides interfaces to store, find, and remove segment files. * It also has a self-cleanup mechanism to clean up stale segment files. It periodically checks the last access time @@ -335,11 +335,11 @@ long addSegment(String supervisorTaskId, String subTaskId, DataSegment segment, } @Nullable - public File findPartitionFile(String supervisorTaskId, String subTaskId, Interval interval, int partitionId) + public File findPartitionFile(String supervisorTaskId, String subTaskId, Interval interval, int bucketId) { TaskIdUtils.validateId("supervisorTaskId", supervisorTaskId); for (StorageLocation location : shuffleDataLocations) { - final File partitionDir = new File(location.getPath(), getPartitionDir(supervisorTaskId, interval, partitionId)); + final File partitionDir = new File(location.getPath(), getPartitionDir(supervisorTaskId, interval, bucketId)); if (partitionDir.exists()) { supervisorTaskCheckTimes.put(supervisorTaskId, getExpiryTimeFromNow()); final File[] segmentFiles = partitionDir.listFiles(); @@ -384,23 +384,23 @@ private static String getPartitionFilePath( String supervisorTaskId, String subTaskId, Interval interval, - int partitionId + int bucketId ) { - return Paths.get(getPartitionDir(supervisorTaskId, interval, partitionId), subTaskId).toString(); + return Paths.get(getPartitionDir(supervisorTaskId, interval, bucketId), subTaskId).toString(); } private static String getPartitionDir( String supervisorTaskId, Interval interval, - int partitionId + int bucketId ) { return Paths.get( supervisorTaskId, interval.getStart().toString(), interval.getEnd().toString(), - String.valueOf(partitionId) + String.valueOf(bucketId) ).toString(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java index d1adcb9f2d22..0e0e9364e211 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/worker/http/ShuffleResource.java @@ -75,7 +75,7 @@ public Response getPartition( @PathParam("subTaskId") String subTaskId, @QueryParam("startTime") String startTime, @QueryParam("endTime") String endTime, - @QueryParam("partitionId") int partitionId + @QueryParam("bucketId") int bucketId ) { final Interval interval = new Interval(DateTimes.of(startTime), DateTimes.of(endTime)); @@ -83,16 +83,16 @@ public Response getPartition( supervisorTaskId, subTaskId, interval, - partitionId + bucketId ); if (partitionFile == null) { final String errorMessage = StringUtils.format( - "Can't find the partition for supervisorTask[%s], subTask[%s], interval[%s], and partitionId[%s]", + "Can't find the partition for supervisorTask[%s], subTask[%s], interval[%s], and bucketId[%s]", supervisorTaskId, subTaskId, interval, - partitionId + bucketId ); return Response.status(Status.NOT_FOUND).entity(errorMessage).build(); } else { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index c5301bf66419..ce1e1fc1fccd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -880,14 +880,14 @@ public void testCannotAddToExistingSingleDimensionShardSpecs() throws Exception .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new SingleDimensionShardSpec("foo", null, "bar", 0)) + .shardSpec(new SingleDimensionShardSpec("foo", null, "bar", 0, 2)) .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new SingleDimensionShardSpec("foo", "bar", null, 1)) + .shardSpec(new SingleDimensionShardSpec("foo", "bar", null, 1, 2)) .size(0) .build() ) @@ -914,14 +914,14 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new HashBasedNumberedShardSpec(0, 2, ImmutableList.of("dim1"), objectMapper)) + .shardSpec(new HashBasedNumberedShardSpec(0, 2, 0, 2, ImmutableList.of("dim1"), objectMapper)) .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) - .shardSpec(new HashBasedNumberedShardSpec(1, 2, ImmutableList.of("dim1"), objectMapper)) + .shardSpec(new HashBasedNumberedShardSpec(1, 2, 1, 2, ImmutableList.of("dim1"), objectMapper)) .size(0) .build() ) @@ -935,7 +935,7 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio "seq", null, true, - new HashBasedNumberedPartialShardSpec(ImmutableList.of("dim1"), 2), + new HashBasedNumberedPartialShardSpec(ImmutableList.of("dim1"), 1, 2), lockGranularity ); final SegmentIdWithShardSpec segmentIdentifier = action.perform(task, taskActionTestKit.getTaskActionToolbox()); @@ -946,7 +946,7 @@ public void testWithPartialShardSpecAndOvershadowingSegments() throws IOExceptio Assert.assertTrue(shardSpec instanceof HashBasedNumberedShardSpec); final HashBasedNumberedShardSpec hashBasedNumberedShardSpec = (HashBasedNumberedShardSpec) shardSpec; - Assert.assertEquals(2, hashBasedNumberedShardSpec.getPartitions()); + Assert.assertEquals(2, hashBasedNumberedShardSpec.getNumCorePartitions()); Assert.assertEquals(ImmutableList.of("dim1"), hashBasedNumberedShardSpec.getPartitionDimensions()); } @@ -1029,10 +1029,7 @@ private void assertSameIdentifier(final SegmentIdWithShardSpec expected, final S if (expected.getShardSpec().getClass() == NumberedShardSpec.class && actual.getShardSpec().getClass() == NumberedShardSpec.class) { - Assert.assertEquals( - ((NumberedShardSpec) expected.getShardSpec()).getPartitions(), - ((NumberedShardSpec) actual.getShardSpec()).getPartitions() - ); + Assert.assertEquals(expected.getShardSpec().getNumCorePartitions(), actual.getShardSpec().getNumCorePartitions()); } else if (expected.getShardSpec().getClass() == LinearShardSpec.class && actual.getShardSpec().getClass() == LinearShardSpec.class) { // do nothing diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java index de3fa29a489b..513a590be9fd 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTaskTest.java @@ -337,6 +337,7 @@ public void testBasics() throws Exception { expectPublishedSegments(1); final AppenderatorDriverRealtimeIndexTask task = makeRealtimeTask(null); + Assert.assertTrue(task.supportsQueries()); final ListenableFuture statusFuture = runTask(task); // Wait for firehose to show up, it starts off null. 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 cf2bb2a255df..e08361450fb9 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 @@ -19,10 +19,10 @@ package org.apache.druid.indexing.common.task; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.jsontype.NamedType; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.io.Files; import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.IndexingServiceClient; @@ -45,6 +45,7 @@ import org.apache.druid.indexing.common.task.CompactionTask.Builder; import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -57,6 +58,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.indexing.DataSchema; @@ -84,6 +86,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -125,19 +128,7 @@ public class CompactionTaskRunTest extends IngestionTestBase false, 0 ); - private static final CompactionState DEFAULT_COMPACTION_STATE = new CompactionState( - new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), - ImmutableMap.of( - "bitmap", - ImmutableMap.of("type", "roaring", "compressRunOnSerialization", true), - "dimensionCompression", - "lz4", - "metricCompression", - "lz4", - "longEncoding", - "longs" - ) - ); + private static CompactionState DEFAULT_COMPACTION_STATE; private static final List TEST_ROWS = ImmutableList.of( "2014-01-01T00:00:10Z,a,1\n", @@ -195,6 +186,17 @@ public Collection fetchUsedSegmentsInDataSourceForIntervals( this.lockGranularity = lockGranularity; } + @BeforeClass + public static void setupClass() throws JsonProcessingException + { + ObjectMapper mapper = new DefaultObjectMapper(); + + DEFAULT_COMPACTION_STATE = new CompactionState( + new DynamicPartitionsSpec(5000000, Long.MAX_VALUE), + mapper.readValue(mapper.writeValueAsString(new IndexSpec()), Map.class) + ); + } + @Before public void setup() throws IOException { @@ -250,7 +252,7 @@ public void testRun() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } @@ -299,7 +301,7 @@ public void testRunCompactionTwice() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } @@ -332,7 +334,7 @@ public void testRunCompactionTwice() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } } @@ -412,7 +414,11 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc for (int i = 0; i < 6; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", 3 + i / 2, 3 + i / 2 + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + if (lockGranularity == LockGranularity.SEGMENT) { + Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + } else { + Assert.assertEquals(new NumberedShardSpec(i % 2, 2), segments.get(i).getShardSpec()); + } } Assert.assertTrue(compactionFuture.get().lhs.isSuccess()); @@ -432,7 +438,7 @@ public void testRunIndexAndCompactAtTheSameTimeForDifferentInterval() throws Exc segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } } @@ -470,7 +476,7 @@ public void testWithSegmentGranularity() throws Exception Assert.assertEquals(1, segments.size()); Assert.assertEquals(Intervals.of("2014-01-01/2014-01-02"), segments.get(0).getInterval()); - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(0).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(0).getShardSpec()); Assert.assertEquals(DEFAULT_COMPACTION_STATE, segments.get(0).getLastCompactionState()); // hour segmentGranularity @@ -488,7 +494,7 @@ public void testWithSegmentGranularity() throws Exception for (int i = 0; i < 3; i++) { Assert.assertEquals(Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1), segments.get(i).getInterval()); - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); Assert.assertEquals(DEFAULT_COMPACTION_STATE, segments.get(i).getLastCompactionState()); } } @@ -592,7 +598,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(i % 2, 2), segments.get(i).getShardSpec()); } } @@ -665,7 +671,7 @@ public void testRunIndexAndCompactForSameSegmentAtTheSameTime2() throws Exceptio segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(i % 2, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(i % 2, 2), segments.get(i).getShardSpec()); } } @@ -755,7 +761,7 @@ public void testRunRegularIndexTaskWithIngestSegmentFirehose() throws Exception segments.get(i).getShardSpec() ); } else { - Assert.assertEquals(new NumberedShardSpec(0, 0), segments.get(i).getShardSpec()); + Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec()); } } } 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 1d4bec4f99e1..66ee8595d25d 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 @@ -73,7 +73,6 @@ import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.ISE; 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.granularity.Granularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity; @@ -160,7 +159,11 @@ public class CompactionTaskTest Intervals.of("2017-03-01/2017-04-01"), Intervals.of("2017-04-01/2017-05-01"), Intervals.of("2017-05-01/2017-06-01"), - Intervals.of("2017-06-01/2017-07-01") + Intervals.of("2017-06-01/2017-07-01"), + // overlapping intervals + Intervals.of("2017-06-01/2017-06-02"), + Intervals.of("2017-06-15/2017-06-16"), + Intervals.of("2017-06-30/2017-07-01") ); private static final Map MIXED_TYPE_COLUMN_MAP = new HashMap<>(); private static final ParallelIndexTuningConfig TUNING_CONFIG = createTuningConfig(); @@ -191,12 +194,17 @@ public static void setupClass() MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-05-01/2017-06-01"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-01/2017-07-01"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-01/2017-06-02"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-15/2017-06-16"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + MIXED_TYPE_COLUMN_MAP.put(Intervals.of("2017-06-30/2017-07-01"), new DoubleDimensionSchema(MIXED_TYPE_COLUMN)); + DIMENSIONS = new HashMap<>(); AGGREGATORS = new ArrayList<>(); DIMENSIONS.put(ColumnHolder.TIME_COLUMN_NAME, new LongDimensionSchema(ColumnHolder.TIME_COLUMN_NAME)); DIMENSIONS.put(TIMESTAMP_COLUMN, new LongDimensionSchema(TIMESTAMP_COLUMN)); - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + int numUmbrellaIntervals = 6; + for (int i = 0; i < numUmbrellaIntervals; i++) { final StringDimensionSchema schema = new StringDimensionSchema( "string_dim_" + i, null, @@ -204,15 +212,15 @@ public static void setupClass() ); DIMENSIONS.put(schema.getName(), schema); } - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + for (int i = 0; i < numUmbrellaIntervals; i++) { final LongDimensionSchema schema = new LongDimensionSchema("long_dim_" + i); DIMENSIONS.put(schema.getName(), schema); } - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + for (int i = 0; i < numUmbrellaIntervals; i++) { final FloatDimensionSchema schema = new FloatDimensionSchema("float_dim_" + i); DIMENSIONS.put(schema.getName(), schema); } - for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { + for (int i = 0; i < numUmbrellaIntervals; i++) { final DoubleDimensionSchema schema = new DoubleDimensionSchema("double_dim_" + i); DIMENSIONS.put(schema.getName(), schema); } @@ -224,14 +232,13 @@ public static void setupClass() AGGREGATORS.add(new DoubleLastAggregatorFactory("agg_4", "double_dim_4")); for (int i = 0; i < SEGMENT_INTERVALS.size(); i++) { - final Interval segmentInterval = Intervals.of(StringUtils.format("2017-0%d-01/2017-0%d-01", (i + 1), (i + 2))); SEGMENT_MAP.put( new DataSegment( DATA_SOURCE, - segmentInterval, - "version", + SEGMENT_INTERVALS.get(i), + "version_" + i, ImmutableMap.of(), - findDimensions(i, segmentInterval), + findDimensions(i, SEGMENT_INTERVALS.get(i)), AGGREGATORS.stream().map(AggregatorFactory::getName).collect(Collectors.toList()), new NumberedShardSpec(0, 1), 0, @@ -285,7 +292,7 @@ private static List findDimensions(int startIndex, Interval segmentInter dimensions.add(TIMESTAMP_COLUMN); for (int i = 0; i < 6; i++) { int postfix = i + startIndex; - postfix = postfix >= 6 ? postfix - 6 : postfix; + postfix = postfix % 6; dimensions.add("string_dim_" + postfix); dimensions.add("long_dim_" + postfix); dimensions.add("float_dim_" + postfix); 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 2cb48e67ba3f..513c13c56e44 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 @@ -229,6 +229,8 @@ public void testDeterminePartitions() throws Exception appenderatorsManager ); + Assert.assertFalse(indexTask.supportsQueries()); + final List segments = runTask(indexTask).rhs; Assert.assertEquals(2, segments.size()); @@ -237,13 +239,13 @@ public void testDeterminePartitions() throws Exception Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(0).getInterval()); Assert.assertEquals(HashBasedNumberedShardSpec.class, segments.get(0).getShardSpec().getClass()); Assert.assertEquals(0, segments.get(0).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(0).getShardSpec()).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) segments.get(0).getShardSpec()).getNumCorePartitions()); Assert.assertEquals("test", segments.get(1).getDataSource()); Assert.assertEquals(Intervals.of("2014/P1D"), segments.get(1).getInterval()); Assert.assertEquals(HashBasedNumberedShardSpec.class, segments.get(1).getShardSpec().getClass()); Assert.assertEquals(1, segments.get(1).getShardSpec().getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) segments.get(1).getShardSpec()).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) segments.get(1).getShardSpec()).getNumCorePartitions()); } @Test @@ -1561,8 +1563,8 @@ public void testCsvWithHeaderOfEmptyColumns() throws Exception Assert.assertTrue( StringUtils.format("Actual dimensions: %s", dimensions), - dimensions.equals(Sets.newHashSet("dim", "column_3")) || - dimensions.equals(Sets.newHashSet("column_2", "column_3")) + dimensions.equals(Sets.newHashSet("column_2")) || + dimensions.equals(Sets.newHashSet("dim", "column_2", "column_3")) ); Assert.assertEquals(Collections.singletonList("val"), segment.getMetrics()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java index 855698df84fc..e841fba0f370 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RangePartitionCachingLocalSegmentAllocatorTest.java @@ -19,6 +19,7 @@ package org.apache.druid.indexing.common.task; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.data.input.InputRow; import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; @@ -30,10 +31,13 @@ import org.apache.druid.java.util.common.DateTimes; 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.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.PartitionBoundaries; -import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.apache.druid.timeline.partition.RangeBucketShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; @@ -81,7 +85,7 @@ public class RangePartitionCachingLocalSegmentAllocatorTest INTERVAL_NORMAL, NORMAL_PARTITIONS ); - private CachingSegmentAllocator target; + private SegmentAllocator target; private SequenceNameFunction sequenceNameFunction; @Rule @@ -104,10 +108,11 @@ public void setup() throws IOException toolbox, DATASOURCE, TASKID, + new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, ImmutableList.of()), new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID), partitionAnalysis ); - sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs()); + sequenceNameFunction = ((CachingLocalSegmentAllocator) target).getSequenceNameFunction(); } @Test @@ -161,37 +166,37 @@ public void getSequenceName() } @SuppressWarnings("SameParameterValue") - private void testAllocate(InputRow row, Interval interval, int partitionNum) + private void testAllocate(InputRow row, Interval interval, int bucketId) { - String partitionEnd = getPartitionEnd(interval, partitionNum); - testAllocate(row, interval, partitionNum, partitionEnd); + String partitionEnd = getPartitionEnd(interval, bucketId); + testAllocate(row, interval, bucketId, partitionEnd); } @Nullable - private static String getPartitionEnd(Interval interval, int partitionNum) + private static String getPartitionEnd(Interval interval, int bucketId) { PartitionBoundaries partitions = INTERVAL_TO_PARTITONS.get(interval); - boolean isLastPartition = (partitionNum + 1) == partitions.size(); - return isLastPartition ? null : partitions.get(partitionNum + 1); + boolean isLastPartition = (bucketId + 1) == partitions.size(); + return isLastPartition ? null : partitions.get(bucketId + 1); } - private void testAllocate(InputRow row, Interval interval, int partitionNum, @Nullable String partitionEnd) + private void testAllocate(InputRow row, Interval interval, int bucketId, @Nullable String partitionEnd) { - String partitionStart = getPartitionStart(interval, partitionNum); - testAllocate(row, interval, partitionNum, partitionStart, partitionEnd); + String partitionStart = getPartitionStart(interval, bucketId); + testAllocate(row, interval, bucketId, partitionStart, partitionEnd); } @Nullable - private static String getPartitionStart(Interval interval, int partitionNum) + private static String getPartitionStart(Interval interval, int bucketId) { - boolean isFirstPartition = partitionNum == 0; - return isFirstPartition ? null : INTERVAL_TO_PARTITONS.get(interval).get(partitionNum); + boolean isFirstPartition = bucketId == 0; + return isFirstPartition ? null : INTERVAL_TO_PARTITONS.get(interval).get(bucketId); } private void testAllocate( InputRow row, Interval interval, - int partitionNum, + int bucketId, @Nullable String partitionStart, @Nullable String partitionEnd ) @@ -200,12 +205,12 @@ private void testAllocate( SegmentIdWithShardSpec segmentIdWithShardSpec = allocate(row, sequenceName); Assert.assertEquals( - SegmentId.of(DATASOURCE, interval, INTERVAL_TO_VERSION.get(interval), partitionNum), + SegmentId.of(DATASOURCE, interval, INTERVAL_TO_VERSION.get(interval), bucketId), segmentIdWithShardSpec.asSegmentId() ); - SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segmentIdWithShardSpec.getShardSpec(); + RangeBucketShardSpec shardSpec = (RangeBucketShardSpec) segmentIdWithShardSpec.getShardSpec(); Assert.assertEquals(PARTITION_DIMENSION, shardSpec.getDimension()); - Assert.assertEquals(partitionNum, shardSpec.getPartitionNum()); + Assert.assertEquals(bucketId, shardSpec.getBucketId()); Assert.assertEquals(partitionStart, shardSpec.getStart()); Assert.assertEquals(partitionEnd, shardSpec.getEnd()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 8cc21d02e19a..12ea2145d582 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -189,6 +189,12 @@ public void testDefaultResource() Assert.assertEquals(task.getId(), task.getTaskResource().getAvailabilityGroup()); } + @Test(timeout = 60_000L) + public void testSupportsQueries() + { + final RealtimeIndexTask task = makeRealtimeTask(null); + Assert.assertTrue(task.supportsQueries()); + } @Test(timeout = 60_000L, expected = ExecutionException.class) public void testHandoffTimeout() throws Exception diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.java new file mode 100644 index 000000000000..0818605fc8ea --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ShardSpecsTest.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.indexing.common.task; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.MapBasedInputRow; +import org.apache.druid.indexing.common.TestUtils; +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.timeline.partition.BucketNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ShardSpecsTest extends IngestionTestBase +{ + private final TestUtils testUtils = new TestUtils(); + private final ObjectMapper jsonMapper = testUtils.getTestObjectMapper(); + + public ShardSpecsTest() + { + } + + @Test + public void testShardSpecSelectionWithNullPartitionDimension() + { + HashBucketShardSpec spec1 = new HashBucketShardSpec(0, 2, null, jsonMapper); + HashBucketShardSpec spec2 = new HashBucketShardSpec(1, 2, null, jsonMapper); + + Map>> shardSpecMap = new HashMap<>(); + shardSpecMap.put(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), ImmutableList.of(spec1, spec2)); + + ShardSpecs shardSpecs = new ShardSpecs(shardSpecMap, Granularities.HOUR); + String visitorId = "visitorId"; + String clientType = "clientType"; + long timestamp1 = DateTimes.of("2014-01-01T00:00:00.000Z").getMillis(); + InputRow row1 = new MapBasedInputRow(timestamp1, + Lists.newArrayList(visitorId, clientType), + ImmutableMap.of(visitorId, "0", clientType, "iphone") + ); + + long timestamp2 = DateTimes.of("2014-01-01T00:30:20.456Z").getMillis(); + InputRow row2 = new MapBasedInputRow(timestamp2, + Lists.newArrayList(visitorId, clientType), + ImmutableMap.of(visitorId, "0", clientType, "iphone") + ); + + long timestamp3 = DateTimes.of("2014-01-01T10:10:20.456Z").getMillis(); + InputRow row3 = new MapBasedInputRow(timestamp3, + Lists.newArrayList(visitorId, clientType), + ImmutableMap.of(visitorId, "0", clientType, "iphone") + ); + + ShardSpec spec3 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row1); + ShardSpec spec4 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row2); + ShardSpec spec5 = shardSpecs.getShardSpec(Intervals.of("2014-01-01T00:00:00.000Z/2014-01-02T00:00:00.000Z"), row3); + + Assert.assertSame(true, spec3 == spec4); + Assert.assertSame(false, spec3 == spec5); + } +} 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 7970e1dd9b7f..64949deefb34 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 @@ -117,31 +117,23 @@ Set runTestTask( maxNumConcurrentSubTasks ); + return runTask(task, expectedTaskStatus); + } + + Set runTask(ParallelIndexSupervisorTask task, TaskState expectedTaskStatus) + { task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); TaskStatus taskStatus = getIndexingServiceClient().runAndWait(task); Assert.assertEquals(expectedTaskStatus, taskStatus.getStatusCode()); return getIndexingServiceClient().getPublishedSegments(task); } - private ParallelIndexSupervisorTask newTask( - @Nullable TimestampSpec timestampSpec, - @Nullable DimensionsSpec dimensionsSpec, - @Nullable InputFormat inputFormat, - @Nullable ParseSpec parseSpec, - Interval interval, - File inputDir, - String filter, + ParallelIndexTuningConfig newTuningConfig( DimensionBasedPartitionsSpec partitionsSpec, int maxNumConcurrentSubTasks ) { - GranularitySpec granularitySpec = new UniformGranularitySpec( - Granularities.DAY, - Granularities.MINUTE, - interval == null ? null : Collections.singletonList(interval) - ); - - ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig( + return new ParallelIndexTuningConfig( null, null, null, @@ -169,6 +161,27 @@ private ParallelIndexSupervisorTask newTask( null, null ); + } + + private ParallelIndexSupervisorTask newTask( + @Nullable TimestampSpec timestampSpec, + @Nullable DimensionsSpec dimensionsSpec, + @Nullable InputFormat inputFormat, + @Nullable ParseSpec parseSpec, + Interval interval, + File inputDir, + String filter, + DimensionBasedPartitionsSpec partitionsSpec, + int maxNumConcurrentSubTasks + ) + { + GranularitySpec granularitySpec = new UniformGranularitySpec( + Granularities.DAY, + Granularities.MINUTE, + interval == null ? null : Collections.singletonList(interval) + ); + + ParallelIndexTuningConfig tuningConfig = newTuningConfig(partitionsSpec, maxNumConcurrentSubTasks); final ParallelIndexIngestionSpec ingestionSpec; @@ -185,9 +198,7 @@ private ParallelIndexSupervisorTask newTask( "dataSource", timestampSpec, dimensionsSpec, - new AggregatorFactory[]{ - new LongSumAggregatorFactory("val", "val") - }, + new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, granularitySpec, null ), 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 ca33815f28eb..470a00372b48 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 @@ -521,7 +521,6 @@ public static void prepareObjectMapper( new NamedType(ParallelIndexSupervisorTask.class, ParallelIndexSupervisorTask.TYPE), new NamedType(SinglePhaseSubTask.class, SinglePhaseSubTask.TYPE), new NamedType(PartialHashSegmentGenerateTask.class, PartialHashSegmentGenerateTask.TYPE), - new NamedType(PartialHashSegmentMergeTask.class, PartialHashSegmentMergeTask.TYPE), new NamedType(PartialRangeSegmentGenerateTask.class, PartialRangeSegmentGenerateTask.TYPE), new NamedType(PartialGenericSegmentMergeTask.class, PartialGenericSegmentMergeTask.TYPE), new NamedType(PartialDimensionDistributionTask.class, PartialDimensionDistributionTask.TYPE) @@ -646,7 +645,7 @@ public > File fetchSegmentFile( supervisorTaskId, location.getSubTaskId(), location.getInterval(), - location.getPartitionId() + location.getBucketId() ); if (zippedFile == null) { throw new ISE("Can't find segment file for location[%s] at path[%s]", location); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java deleted file mode 100644 index 1343b9425833..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReportTest.java +++ /dev/null @@ -1,59 +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.indexing.common.task.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.TestHelper; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class GeneratedHashPartitionsReportTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - - private GeneratedHashPartitionsReport target; - - @Before - public void setup() - { - target = new GeneratedHashPartitionsReport( - "task-id", - Collections.singletonList( - new HashPartitionStat( - ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, - ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID, - ParallelIndexTestingFactory.NUM_ROWS, - ParallelIndexTestingFactory.SIZE_BYTES - ) - ) - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java index 956dbc8fd150..4e46e388f6c6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionLocationTest.java @@ -53,6 +53,6 @@ public void serializesDeserializes() @Test public void hasPartitionIdThatMatchesShardSpec() { - Assert.assertEquals(ParallelIndexTestingFactory.PARTITION_ID, target.getPartitionId()); + Assert.assertEquals(ParallelIndexTestingFactory.PARTITION_ID, target.getBucketId()); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java index 2bcac8edfd47..ffeab43f60d9 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/GenericPartitionStatTest.java @@ -21,10 +21,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.druid.segment.TestHelper; +import org.apache.druid.timeline.partition.HashBucketShardSpec; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.util.Collections; + public class GenericPartitionStatTest { private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); @@ -39,7 +42,12 @@ public void setup() ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, ParallelIndexTestingFactory.USE_HTTPS, ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.HASH_BASED_NUMBERED_SHARD_SPEC, + new HashBucketShardSpec( + ParallelIndexTestingFactory.PARTITION_ID, + ParallelIndexTestingFactory.PARTITION_ID + 1, + Collections.singletonList("dim"), + new ObjectMapper() + ), ParallelIndexTestingFactory.NUM_ROWS, ParallelIndexTestingFactory.SIZE_BYTES ); @@ -54,6 +62,6 @@ public void serializesDeserializes() @Test public void hasPartitionIdThatMatchesSecondaryPartition() { - Assert.assertEquals(target.getSecondaryPartition().getPartitionNum(), target.getPartitionId()); + Assert.assertEquals(target.getSecondaryPartition().getBucketId(), target.getBucketId()); } } 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 new file mode 100644 index 000000000000..e19b208e3691 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionAdjustingCorePartitionSizeTest.java @@ -0,0 +1,165 @@ +/* + * 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.indexing.common.task.batch.parallel; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; +import org.apache.druid.indexer.partitions.HashedPartitionsSpec; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Set; + +@RunWith(Parameterized.class) +public class HashPartitionAdjustingCorePartitionSizeTest extends AbstractMultiPhaseParallelIndexingTest +{ + private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")) + ); + private static final InputFormat INPUT_FORMAT = new CsvInputFormat( + Arrays.asList("ts", "dim1", "dim2", "val"), + null, + false, + false, + 0 + ); + private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M"); + + @Parameterized.Parameters(name = "{0}, maxNumConcurrentSubTasks={1}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK, 2}, + new Object[]{LockGranularity.TIME_CHUNK, 1}, + new Object[]{LockGranularity.SEGMENT, 2} + ); + } + + private final int maxNumConcurrentSubTasks; + + public HashPartitionAdjustingCorePartitionSizeTest(LockGranularity lockGranularity, int maxNumConcurrentSubTasks) + { + super(lockGranularity, true); + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + } + + @Test + public void testLessPartitionsThanBuckets() throws IOException + { + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 3; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "a" + (i + 1), 10 * (i + 1))); + } + } + final DimensionBasedPartitionsSpec partitionsSpec = new HashedPartitionsSpec( + null, + 10, + ImmutableList.of("dim1") + ); + final List segments = new ArrayList<>( + runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, + null, + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS + ) + ); + Assert.assertEquals(3, segments.size()); + segments.sort(Comparator.comparing(segment -> segment.getShardSpec().getPartitionNum())); + int prevPartitionId = -1; + for (DataSegment segment : segments) { + Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); + final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); + Assert.assertEquals(10, shardSpec.getNumBuckets()); + Assert.assertEquals(ImmutableList.of("dim1"), shardSpec.getPartitionDimensions()); + Assert.assertEquals(prevPartitionId + 1, shardSpec.getPartitionNum()); + prevPartitionId = shardSpec.getPartitionNum(); + } + } + + @Test + public void testEqualNumberOfPartitionsToBuckets() throws IOException + { + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 10; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "aa" + (i + 10), 10 * (i + 1))); + } + } + final DimensionBasedPartitionsSpec partitionsSpec = new HashedPartitionsSpec( + null, + 5, + ImmutableList.of("dim1") + ); + final Set segments = runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, + null, + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS + ); + Assert.assertEquals(5, segments.size()); + segments.forEach(segment -> { + Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); + final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(5, shardSpec.getNumCorePartitions()); + Assert.assertEquals(5, shardSpec.getNumBuckets()); + Assert.assertEquals(ImmutableList.of("dim1"), shardSpec.getPartitionDimensions()); + }); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java index 5dc9560d57fd..ef8f09562311 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionCachingLocalSegmentAllocatorTest.java @@ -29,17 +29,19 @@ import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.actions.LockListAction; import org.apache.druid.indexing.common.actions.TaskActionClient; -import org.apache.druid.indexing.common.task.CachingSegmentAllocator; -import org.apache.druid.indexing.common.task.NonLinearlyPartitionedSequenceNameFunction; +import org.apache.druid.indexing.common.task.CachingLocalSegmentAllocator; import org.apache.druid.indexing.common.task.SegmentAllocators; import org.apache.druid.indexing.common.task.SequenceNameFunction; import org.apache.druid.indexing.common.task.SupervisorTaskAccessWithNullClient; import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis; 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.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.appenderator.SegmentAllocator; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.SegmentId; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; import org.easymock.EasyMock; import org.joda.time.Interval; import org.junit.Assert; @@ -69,7 +71,7 @@ public class HashPartitionCachingLocalSegmentAllocatorTest Collections.singletonList(DIMENSION) ); - private CachingSegmentAllocator target; + private SegmentAllocator target; private SequenceNameFunction sequenceNameFunction; @Before @@ -82,10 +84,11 @@ public void setup() throws IOException toolbox, DATASOURCE, TASKID, + new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, ImmutableList.of()), new SupervisorTaskAccessWithNullClient(SUPERVISOR_TASKID), partitionAnalysis ); - sequenceNameFunction = new NonLinearlyPartitionedSequenceNameFunction(TASKID, target.getShardSpecs()); + sequenceNameFunction = ((CachingLocalSegmentAllocator) target).getSequenceNameFunction(); } @Test @@ -100,10 +103,10 @@ public void allocatesCorrectShardSpec() throws IOException SegmentId.of(DATASOURCE, INTERVAL, VERSION, PARTITION_NUM), segmentIdWithShardSpec.asSegmentId() ); - HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segmentIdWithShardSpec.getShardSpec(); + HashBucketShardSpec shardSpec = (HashBucketShardSpec) segmentIdWithShardSpec.getShardSpec(); Assert.assertEquals(PARTITION_DIMENSIONS, shardSpec.getPartitionDimensions()); - Assert.assertEquals(NUM_PARTITONS, shardSpec.getPartitions()); - Assert.assertEquals(PARTITION_NUM, shardSpec.getPartitionNum()); + Assert.assertEquals(NUM_PARTITONS, shardSpec.getNumBuckets()); + Assert.assertEquals(PARTITION_NUM, shardSpec.getBucketId()); } @Test 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 bcd3cbe85842..7fcabd0911ba 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 @@ -75,24 +75,31 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh false, 0 ); - private static final int MAX_NUM_CONCURRENT_SUB_TASKS = 2; private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M"); @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") public static Iterable constructorFeeder() { return ImmutableList.of( - new Object[]{LockGranularity.TIME_CHUNK, false}, - new Object[]{LockGranularity.TIME_CHUNK, true}, - new Object[]{LockGranularity.SEGMENT, true} + new Object[]{LockGranularity.TIME_CHUNK, false, 2}, + new Object[]{LockGranularity.TIME_CHUNK, true, 2}, + new Object[]{LockGranularity.TIME_CHUNK, true, 1}, + new Object[]{LockGranularity.SEGMENT, true, 2} ); } + private final int maxNumConcurrentSubTasks; + private File inputDir; - public HashPartitionMultiPhaseParallelIndexingTest(LockGranularity lockGranularity, boolean useInputFormatApi) + public HashPartitionMultiPhaseParallelIndexingTest( + LockGranularity lockGranularity, + boolean useInputFormatApi, + int maxNumConcurrentSubTasks + ) { super(lockGranularity, useInputFormatApi); + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; } @Before @@ -132,7 +139,7 @@ public void testRun() throws Exception inputDir, "test_*", new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")), - MAX_NUM_CONCURRENT_SUB_TASKS, + maxNumConcurrentSubTasks, TaskState.SUCCESS ); } else { @@ -145,7 +152,7 @@ public void testRun() throws Exception inputDir, "test_*", new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")), - MAX_NUM_CONCURRENT_SUB_TASKS, + maxNumConcurrentSubTasks, TaskState.SUCCESS ); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java deleted file mode 100644 index 1eb6f867cd8e..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HashPartitionStatTest.java +++ /dev/null @@ -1,59 +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.indexing.common.task.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.TestHelper; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class HashPartitionStatTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - - private HashPartitionStat target; - - @Before - public void setup() - { - target = new HashPartitionStat( - ParallelIndexTestingFactory.TASK_EXECUTOR_HOST, - ParallelIndexTestingFactory.TASK_EXECUTOR_PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID, - ParallelIndexTestingFactory.NUM_ROWS, - ParallelIndexTestingFactory.SIZE_BYTES - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } - - @Test - public void hasPartitionIdThatMatchesSecondaryPartition() - { - Assert.assertEquals(target.getSecondaryPartition().intValue(), target.getPartitionId()); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java index 45df76d58da2..3ddb63b65fbe 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/HttpShuffleClientTest.java @@ -203,7 +203,7 @@ private TestPartitionLocation() } @Override - int getPartitionId() + int getBucketId() { return getSecondaryPartition(); } 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 88dac06b89ed..9f8a07dfaebd 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 @@ -677,6 +677,7 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception .getGivenOrDefaultPartitionsSpec(); final SegmentAllocator segmentAllocator = SegmentAllocators.forLinearPartitioning( toolbox, + getId(), new SupervisorTaskAccess(getSupervisorTaskId(), taskClient), getIngestionSchema().getDataSchema(), getTaskLockHelper(), 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 b882aac207ab..3ae79a3b6966 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 @@ -19,9 +19,11 @@ package org.apache.druid.indexing.common.task.batch.parallel; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.hamcrest.Matchers; import org.joda.time.Interval; import org.junit.Assert; @@ -45,8 +47,8 @@ public class ParallelIndexSupervisorTaskTest public static class CreateMergeIoConfigsTest { private static final int TOTAL_NUM_MERGE_TASKS = 10; - private static final Function, PartialHashSegmentMergeIOConfig> - CREATE_PARTIAL_SEGMENT_MERGE_IO_CONFIG = PartialHashSegmentMergeIOConfig::new; + private static final Function, PartialGenericSegmentMergeIOConfig> + CREATE_PARTIAL_SEGMENT_MERGE_IO_CONFIG = PartialGenericSegmentMergeIOConfig::new; @Parameterized.Parameters(name = "count = {0}") public static Iterable data() @@ -66,14 +68,14 @@ public static Iterable data() @Test public void handlesLastPartitionCorrectly() { - List assignedPartitionLocation = createMergeIOConfigs(); + List assignedPartitionLocation = createMergeIOConfigs(); assertNoMissingPartitions(count, assignedPartitionLocation); } @Test public void sizesPartitionsEvenly() { - List assignedPartitionLocation = createMergeIOConfigs(); + List assignedPartitionLocation = createMergeIOConfigs(); List actualPartitionSizes = assignedPartitionLocation.stream() .map(i -> i.getPartitionLocations().size()) .collect(Collectors.toList()); @@ -89,7 +91,7 @@ public void sizesPartitionsEvenly() ); } - private List createMergeIOConfigs() + private List createMergeIOConfigs() { return ParallelIndexSupervisorTask.createMergeIOConfigs( TOTAL_NUM_MERGE_TASKS, @@ -98,7 +100,7 @@ private List createMergeIOConfigs() ); } - private static Map, List> createPartitionToLocations(int count) + private static Map, List> createPartitionToLocations(int count) { return IntStream.range(0, count).boxed().collect( Collectors.toMap( @@ -108,15 +110,15 @@ private static Map, List> createP ); } - private static HashPartitionLocation createPartitionLocation(int id) + private static GenericPartitionLocation createPartitionLocation(int id) { - return new HashPartitionLocation( + return new GenericPartitionLocation( "host", 0, false, "subTaskId", createInterval(id), - id + new BuildingHashBasedNumberedShardSpec(id, id, id + 1, null, new ObjectMapper()) ); } @@ -127,7 +129,7 @@ private static Interval createInterval(int id) private static void assertNoMissingPartitions( int count, - List assignedPartitionLocation + List assignedPartitionLocation ) { List expectedIds = IntStream.range(0, count).boxed().collect(Collectors.toList()); @@ -136,7 +138,7 @@ private static void assertNoMissingPartitions( .flatMap( i -> i.getPartitionLocations() .stream() - .map(HashPartitionLocation::getPartitionId) + .map(GenericPartitionLocation::getBucketId) ) .sorted() .collect(Collectors.toList()); 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 08dd92c5f67a..de7ee8b6cc40 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 @@ -44,7 +44,7 @@ import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.transform.TransformSpec; -import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.easymock.EasyMock; import org.joda.time.Duration; import org.joda.time.Interval; @@ -100,7 +100,8 @@ public > File fetchSegmentFile( private static final String SCHEMA_DIMENSION = "dim"; private static final String DATASOURCE = "datasource"; - static final HashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new HashBasedNumberedShardSpec( + static final BuildingHashBasedNumberedShardSpec HASH_BASED_NUMBERED_SHARD_SPEC = new BuildingHashBasedNumberedShardSpec( + PARTITION_ID, PARTITION_ID, PARTITION_ID + 1, Collections.singletonList("dim"), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java deleted file mode 100644 index 413c34d9d668..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfigTest.java +++ /dev/null @@ -1,54 +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.indexing.common.task.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.segment.TestHelper; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class PartialHashSegmentMergeIOConfigTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - ParallelIndexTestingFactory.HOST, - ParallelIndexTestingFactory.PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.SUBTASK_ID, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID - ); - - private PartialHashSegmentMergeIOConfig target; - - @Before - public void setup() - { - target = new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java deleted file mode 100644 index d734739a0146..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpecTest.java +++ /dev/null @@ -1,68 +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.indexing.common.task.batch.parallel; - -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.segment.TestHelper; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class PartialHashSegmentMergeIngestionSpecTest -{ - private static final ObjectMapper OBJECT_MAPPER = ParallelIndexTestingFactory.createObjectMapper(); - private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - ParallelIndexTestingFactory.HOST, - ParallelIndexTestingFactory.PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.SUBTASK_ID, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID - ); - private static final PartialHashSegmentMergeIOConfig IO_CONFIG = - new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); - private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( - null, - 1, - Collections.emptyList() - ); - - private PartialHashSegmentMergeIngestionSpec target; - - @Before - public void setup() - { - target = new PartialHashSegmentMergeIngestionSpec( - ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), - IO_CONFIG, - new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(PARTITIONS_SPEC) - .build() - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(OBJECT_MAPPER, target); - } -} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java deleted file mode 100644 index d6fe0bb864c4..000000000000 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeTaskTest.java +++ /dev/null @@ -1,88 +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.indexing.common.task.batch.parallel; - -import org.apache.druid.indexer.partitions.HashedPartitionsSpec; -import org.apache.druid.segment.TestHelper; -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import java.util.Collections; - -public class PartialHashSegmentMergeTaskTest extends AbstractParallelIndexSupervisorTaskTest -{ - private static final HashPartitionLocation HASH_PARTITION_LOCATION = new HashPartitionLocation( - ParallelIndexTestingFactory.HOST, - ParallelIndexTestingFactory.PORT, - ParallelIndexTestingFactory.USE_HTTPS, - ParallelIndexTestingFactory.SUBTASK_ID, - ParallelIndexTestingFactory.INTERVAL, - ParallelIndexTestingFactory.PARTITION_ID - ); - private static final PartialHashSegmentMergeIOConfig IO_CONFIG = - new PartialHashSegmentMergeIOConfig(Collections.singletonList(HASH_PARTITION_LOCATION)); - private static final HashedPartitionsSpec PARTITIONS_SPEC = new HashedPartitionsSpec( - null, - 1, - Collections.emptyList() - ); - private static final PartialHashSegmentMergeIngestionSpec INGESTION_SPEC = - new PartialHashSegmentMergeIngestionSpec( - ParallelIndexTestingFactory.createDataSchema(ParallelIndexTestingFactory.INPUT_INTERVALS), - IO_CONFIG, - new ParallelIndexTestingFactory.TuningConfigBuilder() - .partitionsSpec(PARTITIONS_SPEC) - .build() - ); - - private PartialHashSegmentMergeTask target; - - @Before - public void setup() - { - target = new PartialHashSegmentMergeTask( - ParallelIndexTestingFactory.AUTOMATIC_ID, - ParallelIndexTestingFactory.GROUP_ID, - ParallelIndexTestingFactory.TASK_RESOURCE, - ParallelIndexTestingFactory.SUPERVISOR_TASK_ID, - ParallelIndexTestingFactory.NUM_ATTEMPTS, - INGESTION_SPEC, - ParallelIndexTestingFactory.CONTEXT, - ParallelIndexTestingFactory.INDEXING_SERVICE_CLIENT, - ParallelIndexTestingFactory.TASK_CLIENT_FACTORY, - ParallelIndexTestingFactory.SHUFFLE_CLIENT - ); - } - - @Test - public void serializesDeserializes() - { - TestHelper.testSerializesDeserializes(getObjectMapper(), target); - } - - @Test - public void hasCorrectPrefixForAutomaticId() - { - String id = target.getId(); - Assert.assertThat(id, Matchers.startsWith(PartialHashSegmentMergeTask.TYPE)); - } -} 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 new file mode 100644 index 000000000000..2e4aa44dc5f7 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/RangePartitionAdjustingCorePartitionSizeTest.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.indexing.common.task.batch.parallel; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.impl.CsvInputFormat; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec; +import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.File; +import java.io.IOException; +import java.io.Writer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; + +@RunWith(Parameterized.class) +public class RangePartitionAdjustingCorePartitionSizeTest extends AbstractMultiPhaseParallelIndexingTest +{ + private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null); + private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2")) + ); + private static final InputFormat INPUT_FORMAT = new CsvInputFormat( + Arrays.asList("ts", "dim1", "dim2", "val"), + null, + false, + false, + 0 + ); + private static final Interval INTERVAL_TO_INDEX = Intervals.of("2020-01-01/P1M"); + + @Parameterized.Parameters(name = "{0}, maxNumConcurrentSubTasks={1}") + public static Iterable constructorFeeder() + { + return ImmutableList.of( + new Object[]{LockGranularity.TIME_CHUNK, 2}, + new Object[]{LockGranularity.TIME_CHUNK, 1}, + new Object[]{LockGranularity.SEGMENT, 2} + ); + } + + private final int maxNumConcurrentSubTasks; + + public RangePartitionAdjustingCorePartitionSizeTest(LockGranularity lockGranularity, int maxNumConcurrentSubTasks) + { + super(lockGranularity, true); + this.maxNumConcurrentSubTasks = maxNumConcurrentSubTasks; + } + + @Test + public void testLessPartitionsThanBuckets() throws IOException + { + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 2; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,aaa,b1,10\n")); + } + } + for (int i = 0; i < 3; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + (i + 2)).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,zzz,b1,10\n")); + } + } + final DimensionBasedPartitionsSpec partitionsSpec = new SingleDimensionPartitionsSpec( + 2, + null, + "dim1", + false + ); + final List segments = new ArrayList<>( + runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, + null, + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS + ) + ); + Assert.assertEquals(1, segments.size()); + final DataSegment segment = segments.get(0); + Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); + final SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segment.getShardSpec(); + Assert.assertEquals(1, shardSpec.getNumCorePartitions()); + Assert.assertEquals(0, shardSpec.getPartitionNum()); + Assert.assertEquals("dim1", shardSpec.getDimension()); + } + + @Test + public void testEqualNumberOfPartitionsToBuckets() throws IOException + { + final File inputDir = temporaryFolder.newFolder(); + for (int i = 0; i < 10; i++) { + try (final Writer writer = + Files.newBufferedWriter(new File(inputDir, "test_" + i).toPath(), StandardCharsets.UTF_8)) { + writer.write(StringUtils.format("2020-01-01T00:00:00,%s,b1,%d\n", "aa" + (i + 10), 10 * (i + 1))); + } + } + final DimensionBasedPartitionsSpec partitionsSpec = new SingleDimensionPartitionsSpec( + 2, + null, + "dim1", + false + ); + final Set segments = runTestTask( + TIMESTAMP_SPEC, + DIMENSIONS_SPEC, + INPUT_FORMAT, + null, + INTERVAL_TO_INDEX, + inputDir, + "test_*", + partitionsSpec, + maxNumConcurrentSubTasks, + TaskState.SUCCESS + ); + Assert.assertEquals(5, segments.size()); + segments.forEach(segment -> { + Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); + final SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segment.getShardSpec(); + Assert.assertEquals(5, shardSpec.getNumCorePartitions()); + Assert.assertTrue(shardSpec.getPartitionNum() < shardSpec.getNumCorePartitions()); + Assert.assertEquals("dim1", shardSpec.getDimension()); + }); + } +} 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 7b1baa9bc1f0..4abb539f333c 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,6 +43,8 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -57,10 +59,13 @@ import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Set; @@ -155,6 +160,19 @@ private void runTestTask(@Nullable Interval interval, Granularity segmentGranula final ParallelIndexSupervisorTask task = newTask(interval, segmentGranularity, appendToExisting, true); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); + assertShardSpec(interval, appendToExisting); + } + + private void runOverwriteTask( + @Nullable Interval interval, + Granularity segmentGranularity, + LockGranularity actualLockGranularity + ) + { + final ParallelIndexSupervisorTask task = newTask(interval, segmentGranularity, false, true); + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); + Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); + assertShardSpecAfterOverwrite(interval, actualLockGranularity); } private void runTestTask(@Nullable Interval interval, Granularity segmentGranularity) @@ -173,7 +191,11 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s ); // Reingest the same data. Each segment should get replaced by a segment with a newer version. - runTestTask(inputInterval, secondSegmentGranularity); + runOverwriteTask( + inputInterval, + secondSegmentGranularity, + secondSegmentGranularity.equals(Granularities.DAY) ? lockGranularity : LockGranularity.TIME_CHUNK + ); // Verify that the segment has been replaced. final Collection newSegments = @@ -184,6 +206,62 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s Assert.assertEquals(new HashSet<>(newSegments), visibles); } + private void assertShardSpec(@Nullable Interval interval, boolean appendToExisting) + { + final Interval nonNullInterval = interval == null ? Intervals.ETERNITY : interval; + final Collection segments = + getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", nonNullInterval, Segments.ONLY_VISIBLE); + if (!appendToExisting && lockGranularity != LockGranularity.SEGMENT) { + // Check the core partition set in the shardSpec + final Map> intervalToSegments = new HashMap<>(); + segments.forEach( + segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + for (List segmentsPerInterval : intervalToSegments.values()) { + for (DataSegment segment : segmentsPerInterval) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getNumCorePartitions()); + } + } + } else { + for (DataSegment segment : segments) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); + } + } + } + + private void assertShardSpecAfterOverwrite(@Nullable Interval interval, LockGranularity actualLockGranularity) + { + final Interval nonNullInterval = interval == null ? Intervals.ETERNITY : interval; + final Collection segments = + getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", nonNullInterval, Segments.ONLY_VISIBLE); + final Map> intervalToSegments = new HashMap<>(); + segments.forEach( + segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + if (actualLockGranularity != LockGranularity.SEGMENT) { + // Check the core partition set in the shardSpec + for (List segmentsPerInterval : intervalToSegments.values()) { + for (DataSegment segment : segmentsPerInterval) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getNumCorePartitions()); + } + } + } else { + for (List segmentsPerInterval : intervalToSegments.values()) { + for (DataSegment segment : segmentsPerInterval) { + Assert.assertSame(NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass()); + final NumberedOverwriteShardSpec shardSpec = (NumberedOverwriteShardSpec) segment.getShardSpec(); + Assert.assertEquals(segmentsPerInterval.size(), shardSpec.getAtomicUpdateGroupSize()); + } + } + } + } + @Test public void testWithoutInterval() { @@ -213,9 +291,12 @@ public void testRunInParallelWithDifferentSegmentGranularity() @Test public void testRunInSequential() { - final ParallelIndexSupervisorTask task = newTask(Intervals.of("2017-12/P1M"), false, false); + final Interval interval = Intervals.of("2017-12/P1M"); + final boolean appendToExisting = false; + final ParallelIndexSupervisorTask task = newTask(interval, appendToExisting, false); task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); + assertShardSpec(interval, appendToExisting); } @Test @@ -229,10 +310,12 @@ public void testPublishEmptySegments() @Test public void testWith1MaxNumConcurrentSubTasks() { + final Interval interval = Intervals.of("2017-12/P1M"); + final boolean appendToExisting = false; final ParallelIndexSupervisorTask task = newTask( - Intervals.of("2017-12/P1M"), + interval, Granularities.DAY, - false, + appendToExisting, true, new ParallelIndexTuningConfig( null, @@ -266,6 +349,7 @@ public void testWith1MaxNumConcurrentSubTasks() task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); Assert.assertEquals(TaskState.SUCCESS, getIndexingServiceClient().runAndWait(task).getStatusCode()); Assert.assertNull("Runner must be null if the task was in the sequential mode", task.getCurrentRunner()); + assertShardSpec(interval, appendToExisting); } @Test diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/GeneratorInputSourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/GeneratorInputSourceTest.java new file mode 100644 index 000000000000..9b6ae0d3e36f --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/GeneratorInputSourceTest.java @@ -0,0 +1,165 @@ +/* + * 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.indexing.input; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.generator.DataGenerator; +import org.apache.druid.segment.generator.GeneratorBasicSchemas; +import org.apache.druid.segment.generator.GeneratorColumnSchema; +import org.junit.Assert; +import org.junit.Test; + +import java.io.IOException; + +public class GeneratorInputSourceTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + GeneratorInputSource inputSource = new GeneratorInputSource( + "basic", + null, + 1000, + 2, + 1024L, + DateTimes.nowUtc().getMillis(), + 1000, + 1.0 + ); + + String serialized = MAPPER.writeValueAsString(inputSource); + GeneratorInputSource sauce = MAPPER.readValue(serialized, GeneratorInputSource.class); + + Assert.assertEquals(inputSource, sauce); + } + + @Test + public void testSerdeWithSchema() throws JsonProcessingException + { + GeneratorInputSource inputSource = new GeneratorInputSource( + null, + ImmutableList.of( + GeneratorColumnSchema.makeLazyZipf( + "test", + ValueType.LONG, + false, + 1, + 0.0, + 0, + 1000, + 1.3 + ) + ), + 1000, + 2, + 1024L, + DateTimes.nowUtc().getMillis(), + 1000, + 1.0 + ); + + String serialized = MAPPER.writeValueAsString(inputSource); + GeneratorInputSource sauce = MAPPER.readValue(serialized, GeneratorInputSource.class); + + Assert.assertEquals(inputSource, sauce); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(GeneratorInputSource.class).usingGetClass().verify(); + } + + @Test + public void testReader() throws IOException + { + final long seed = 1024L; + final long millis = DateTimes.nowUtc().getMillis(); + final int numConsecutiveTimestamps = 1000; + final double timestampIncrement = 1.0; + final int numRows = 1000; + GeneratorInputSource inputSource = new GeneratorInputSource( + "basic", + null, + numRows, + 2, + seed, + millis, + numConsecutiveTimestamps, + timestampIncrement + ); + + DataGenerator generator = new DataGenerator( + GeneratorBasicSchemas.SCHEMA_MAP.get("basic").getColumnSchemas(), + seed, + millis, + numConsecutiveTimestamps, + timestampIncrement + ); + + InputSourceReader reader = inputSource.fixedFormatReader(null, null); + CloseableIterator iterator = reader.read(); + + InputRow first = iterator.next(); + InputRow generatorFirst = generator.nextRow(); + Assert.assertEquals(generatorFirst, first); + Assert.assertTrue(iterator.hasNext()); + int i; + for (i = 1; iterator.hasNext(); i++) { + iterator.next(); + } + Assert.assertEquals(numRows, i); + } + + @Test + public void testSplits() + { + GeneratorInputSource inputSource = new GeneratorInputSource( + "basic", + null, + 1000, + 2, + 1024L, + DateTimes.nowUtc().getMillis(), + 1000, + 1.0 + ); + + Assert.assertEquals(2, inputSource.estimateNumSplits(null, null)); + Assert.assertEquals(false, inputSource.needsFormat()); + Assert.assertEquals(2, inputSource.createSplits(null, null).count()); + Assert.assertEquals( + new Long(2048L), + ((GeneratorInputSource) inputSource.withSplit(new InputSplit<>(2048L))).getSeed() + ); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java index 00f76a48952a..ae8fc3b64fe1 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockboxTest.java @@ -971,8 +971,8 @@ public void testRequestForNewSegmentWithHashPartition() final Task task = NoopTask.create(); lockbox.add(task); - allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedPartialShardSpec(null, 3)); - allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedPartialShardSpec(null, 5)); + allocateSegmentsAndAssert(task, "seq", 3, new HashBasedNumberedPartialShardSpec(null, 1, 3)); + allocateSegmentsAndAssert(task, "seq2", 5, new HashBasedNumberedPartialShardSpec(null, 3, 5)); } private void allocateSegmentsAndAssert( diff --git a/integration-tests/README.md b/integration-tests/README.md index 7fe1bc2b5daa..d5476cda8a39 100644 --- a/integration-tests/README.md +++ b/integration-tests/README.md @@ -68,16 +68,59 @@ can either be 8 or 11. Druid's configuration (using Docker) can be overrided by providing -Doverride.config.path=. The file must contain one property per line, the key must start with `druid_` and the format should be snake case. +## Docker compose + +Docker compose yamls located in "docker" folder + +docker-compose.base.yml - Base file that defines all containers for integration test + +docker-compose.yml - Defines minimal Druid cluster that can be used for non cluster tests + + docker-compose -f docker-compose.yml up + // DRUID_INTEGRATION_TEST_GROUP - this variable is used in Druid docker container for "security" and "query" test group. Use next docker-compose if you want to run security/query tests. + DRUID_INTEGRATION_TEST_GROUP=security docker-compose -f docker-compose.yml up + +docker-compose.override-env.yml - the same configuration as docker-compose.yml + override-env variable that needed to run cloud tests + + // OVERRIDE_ENV - variable that must contains path to Druid configuration file + OVERRIDE_ENV=./environment-configs/override-examples/s3 docker-compose -f docker-compose.override-env.yml up + +docker-compose.druid-hadoop.yml - for starting Apache Hadoop 2.8.5 cluster with the same setup as the Druid tutorial + + docker-compose -f docker-compose.druid-hadoop.yml up + +## Manual bringing up docker containers and running tests + +1. Build druid-cluster, druid-hadoop docker images. From root module run maven command: +``` +mvn clean install -pl integration-tests -P integration-tests -Ddocker.run.skip=true -Dmaven.test.skip=true +``` + +2. Run druid cluster by docker-compose: + +``` +- Basic Druid cluster: +docker-compose -f integration-tests/docker/docker-compose.yml up +- Druid cluster with override env for cloud integration tests: +OVERRIDE_ENV= docker-compose -f ${DOCKERDIR}/docker-compose.override-env.yml up +- Druid hadoop: +docker-compose -f ${DOCKERDIR}/docker-compose.druid-hadoop.yml up +``` + +3. Run maven command to execute tests with -Ddocker.build.skip=true -Ddocker.run.skip=true + ## Tips & tricks for debugging and developing integration tests ### Useful mvn command flags -- -Dskip.start.docker=true to skip starting docker containers. This can save ~3 minutes by skipping building and bringing +- -Ddocker.build.skip=true to skip build druid containers. +If you do not apply any change to druid then you can do not rebuild druid. +This can save ~4 minutes to build druid cluster and druid hadoop. +You need to build druid containers only once, after you can skip docker build step. +- -Ddocker.run.skip=true to skip starting docker containers. This can save ~3 minutes by skipping building and bringing up the docker containers (Druid, Kafka, Hadoop, MYSQL, zookeeper, etc). Please make sure that you actually do have these containers already running if using this flag. Additionally, please make sure that the running containers are in the same state that the setup script (run_cluster.sh) would have brought it up in. -- -Dskip.stop.docker=true to skip stopping and teardowning down the docker containers. This can be useful in further -debugging after the integration tests have finish running. ### Debugging Druid while running tests diff --git a/integration-tests/build_run_cluster.sh b/integration-tests/build_run_cluster.sh new file mode 100755 index 000000000000..ced683eef9f1 --- /dev/null +++ b/integration-tests/build_run_cluster.sh @@ -0,0 +1,39 @@ +#!/usr/bin/env bash +# 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. + +echo $DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH + +export DIR=$(cd $(dirname $0) && pwd) +export HADOOP_DOCKER_DIR=$DIR/../examples/quickstart/tutorial/hadoop/docker +export DOCKERDIR=$DIR/docker +export SHARED_DIR=${HOME}/shared + +# so docker IP addr will be known during docker build +echo ${DOCKER_IP:=127.0.0.1} > $DOCKERDIR/docker_ip + +if !($DRUID_INTEGRATION_TEST_SKIP_BUILD_DOCKER); then + bash ./script/copy_resources.sh + bash ./script/docker_build_containers.sh +fi + +if !($DRUID_INTEGRATION_TEST_SKIP_RUN_DOCKER); then + bash ./stop_cluster.sh + bash ./script/docker_run_cluster.sh +fi + +if ($DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER); then + bash ./script/copy_hadoop_resources.sh +fi diff --git a/integration-tests/docker-base/setup.sh b/integration-tests/docker-base/setup.sh index a6dc55283a94..7a066ec93c97 100644 --- a/integration-tests/docker-base/setup.sh +++ b/integration-tests/docker-base/setup.sh @@ -31,18 +31,26 @@ apt-get install -y mysql-server apt-get install -y supervisor # Zookeeper -wget -q -O /tmp/zookeeper-3.4.14.tar.gz "https://apache.org/dist/zookeeper/zookeeper-3.4.14/zookeeper-3.4.14.tar.gz" -tar -xzf /tmp/zookeeper-3.4.14.tar.gz -C /usr/local -cp /usr/local/zookeeper-3.4.14/conf/zoo_sample.cfg /usr/local/zookeeper-3.4.14/conf/zoo.cfg -ln -s /usr/local/zookeeper-3.4.14 /usr/local/zookeeper -rm /tmp/zookeeper-3.4.14.tar.gz + +#ZK_VERSION=3.5.8 +#ZK_TAR=apache-zookeeper-$ZK_VERSION-bin + +ZK_VERISON=3.4.14 +ZK_TAR=zookeeper-$ZK_VERSION + +wget -q -O /tmp/$ZK_TAR.tar.gz "https://apache.org/dist/zookeeper/zookeeper-$ZK_VERSION/$ZK_TAR.tar.gz" +tar -xzf /tmp/$ZK_TAR.tar.gz -C /usr/local +cp /usr/local/$ZK_TAR/conf/zoo_sample.cfg /usr/local/$ZK_TAR/conf/zoo.cfg +ln -s /usr/local/$ZK_TAR /usr/local/zookeeper +rm /tmp/$ZK_TAR.tar.gz # Kafka # Match the version to the Kafka client used by KafkaSupervisor -wget -q -O /tmp/kafka_2.12-2.1.1.tgz "https://apache.org/dist/kafka/2.1.1/kafka_2.12-2.1.1.tgz" -tar -xzf /tmp/kafka_2.12-2.1.1.tgz -C /usr/local -ln -s /usr/local/kafka_2.12-2.1.1 /usr/local/kafka -rm /tmp/kafka_2.12-2.1.1.tgz +KAFKA_VERSION=2.5.0 +wget -q -O /tmp/kafka_2.12-$KAFKA_VERSION.tgz "https://apache.org/dist/kafka/$KAFKA_VERSION/kafka_2.12-$KAFKA_VERSION.tgz" +tar -xzf /tmp/kafka_2.12-$KAFKA_VERSION.tgz -C /usr/local +ln -s /usr/local/kafka_2.12-$KAFKA_VERSION /usr/local/kafka +rm /tmp/kafka_2.12-$KAFKA_VERSION.tgz # Druid system user adduser --system --group --no-create-home druid \ diff --git a/integration-tests/docker/docker-compose.base.yml b/integration-tests/docker/docker-compose.base.yml new file mode 100644 index 000000000000..1f7931a623af --- /dev/null +++ b/integration-tests/docker/docker-compose.base.yml @@ -0,0 +1,271 @@ +# 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. + +version: "2.2" +services: + druid-it-hadoop: + image: druid-it/hadoop:2.8.5 + container_name: druid-it-hadoop + ports: + - 2049:2049 + - 2122:2122 + - 8020:8020 + - 8021:8021 + - 8030:8030 + - 8031:8031 + - 8032:8032 + - 8033:8033 + - 8040:8040 + - 8042:8042 + - 8088:8088 + - 8443:8443 + - 9000:9000 + - 10020:10020 + - 19888:19888 + - 34455:34455 + - 50010:50010 + - 50020:50020 + - 50030:50030 + - 50060:50060 + - 50070:50070 + - 50075:50075 + - 50090:50090 + - 51111:51111 + networks: + druid-it-net: + ipv4_address: 172.172.172.13 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./../src/test/resources:/resources + hostname: "druid-it-hadoop" + command: "bash -c 'echo Start druid-it-hadoop container... && \ + /etc/bootstrap.sh && \ + tail -f /dev/null'" + + druid-zookeeper-kafka: + image: druid/cluster + container_name: druid-zookeeper-kafka + ports: + - 2181:2181 + - 9092:9092 + - 9093:9093 + networks: + druid-it-net: + ipv4_address: 172.172.172.2 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/zookeeper.conf:/usr/lib/druid/conf/zookeeper.conf + - ./service-supervisords/kafka.conf:/usr/lib/druid/conf/kafka.conf + env_file: + - ./environment-configs/common + + druid-metadata-storage: + image: druid/cluster + container_name: druid-metadata-storage + ports: + - 3306:3306 + networks: + druid-it-net: + ipv4_address: 172.172.172.3 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/metadata-storage.conf:/usr/lib/druid/conf/metadata-storage.conf + env_file: + - ./environment-configs/common + + druid-overlord: + image: druid/cluster + container_name: druid-overlord + networks: + druid-it-net: + ipv4_address: 172.172.172.4 + ports: + - 8090:8090 + - 8290:8290 + - 5009:5009 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/overlord + + druid-coordinator: + image: druid/cluster + container_name: druid-coordinator + networks: + druid-it-net: + ipv4_address: 172.172.172.5 + ports: + - 8081:8081 + - 8281:8281 + - 5006:5006 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid-overlord.conf + env_file: + - ./environment-configs/common + - ./environment-configs/coordinator + + druid-historical: + image: druid/cluster + container_name: druid-historical + networks: + druid-it-net: + ipv4_address: 172.172.172.6 + ports: + - 8083:8083 + - 8283:8283 + - 5007:5007 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/historical + + druid-middlemanager: + image: druid/cluster + container_name: druid-middlemanager + networks: + druid-it-net: + ipv4_address: 172.172.172.7 + ports: + - 5008:5008 + - 8091:8091 + - 8291:8291 + - 8100:8100 + - 8101:8101 + - 8102:8102 + - 8103:8103 + - 8104:8104 + - 8105:8105 + - 8300:8300 + - 8301:8301 + - 8302:8302 + - 8303:8303 + - 8304:8304 + - 8305:8305 + privileged: true + volumes: + - ./../src/test/resources:/resources + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/middlemanager + + druid-broker: + image: druid/cluster + container_name: druid-broker + networks: + druid-it-net: + ipv4_address: 172.172.172.8 + ports: + - 5005:5005 + - 8082:8082 + - 8282:8282 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/broker + + druid-router: + image: druid/cluster + container_name: druid-router + networks: + druid-it-net: + ipv4_address: 172.172.172.9 + ports: + - 5004:5004 + - 8888:8888 + - 9088:9088 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/router + + druid-router-permissive-tls: + image: druid/cluster + container_name: druid-router-permissive-tls + networks: + druid-it-net: + ipv4_address: 172.172.172.10 + ports: + - 5001:5001 + - 8889:8889 + - 9089:9089 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/router-permissive-tls + + druid-router-no-client-auth-tls: + image: druid/cluster + container_name: druid-router-no-client-auth-tls + networks: + druid-it-net: + ipv4_address: 172.172.172.11 + ports: + - 5002:5002 + - 8890:8890 + - 9090:9090 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/router-no-client-auth-tls + + druid-router-custom-check-tls: + image: druid/cluster + container_name: druid-router-custom-check-tls + networks: + druid-it-net: + ipv4_address: 172.172.172.12 + ports: + - 5003:5003 + - 8891:8891 + - 9091:9091 + privileged: true + volumes: + - ${HOME}/shared:/shared + - ./service-supervisords/druid.conf:/usr/lib/druid/conf/druid.conf + env_file: + - ./environment-configs/common + - ./environment-configs/router-custom-check-tls + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 \ No newline at end of file diff --git a/integration-tests/docker/docker-compose.druid-hadoop.yml b/integration-tests/docker/docker-compose.druid-hadoop.yml new file mode 100644 index 000000000000..735f9670277b --- /dev/null +++ b/integration-tests/docker/docker-compose.druid-hadoop.yml @@ -0,0 +1,28 @@ +# 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. + +version: "2.2" +services: + druid-it-hadoop: + extends: + file: docker-compose.base.yml + service: druid-it-hadoop + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 \ No newline at end of file diff --git a/integration-tests/docker/docker-compose.override-env.yml b/integration-tests/docker/docker-compose.override-env.yml new file mode 100644 index 000000000000..344ee0385eb6 --- /dev/null +++ b/integration-tests/docker/docker-compose.override-env.yml @@ -0,0 +1,195 @@ +# 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. + +version: "2.2" +services: + druid-zookeeper-kafka: + extends: + file: docker-compose.base.yml + service: druid-zookeeper-kafka + + druid-metadata-storage: + extends: + file: docker-compose.base.yml + service: druid-metadata-storage + depends_on: + - druid-zookeeper-kafka + + druid-overlord: + extends: + file: docker-compose.base.yml + service: druid-overlord + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-metadata-storage:druid-metadata-storage + - druid-zookeeper-kafka:druid-zookeeper-kafka + depends_on: + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-coordinator: + extends: + file: docker-compose.base.yml + service: druid-coordinator + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-overlord:druid-overlord + - druid-metadata-storage:druid-metadata-storage + - druid-zookeeper-kafka:druid-zookeeper-kafka + depends_on: + - druid-overlord + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-historical: + extends: + file: docker-compose.base.yml + service: druid-historical + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + depends_on: + - druid-zookeeper-kafka + + druid-middlemanager: + extends: + file: docker-compose.base.yml + service: druid-middlemanager + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-overlord:druid-overlord + depends_on: + - druid-zookeeper-kafka + - druid-overlord + + druid-broker: + extends: + file: docker-compose.base.yml + service: druid-broker + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-middlemanager:druid-middlemanager + - druid-historical:druid-historical + depends_on: + - druid-zookeeper-kafka + - druid-middlemanager + - druid-historical + + druid-router: + extends: + file: docker-compose.base.yml + service: druid-router + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-coordinator + - druid-broker + + druid-router-permissive-tls: + extends: + file: docker-compose.base.yml + service: druid-router-permissive-tls + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-metadata-storage + - druid-overlord + - druid-coordinator + - druid-historical + - druid-middlemanager + - druid-broker + - druid-router + + druid-router-no-client-auth-tls: + extends: + file: docker-compose.base.yml + service: druid-router-no-client-auth-tls + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-metadata-storage + - druid-overlord + - druid-coordinator + - druid-historical + - druid-middlemanager + - druid-broker + - druid-router + + druid-router-custom-check-tls: + extends: + file: docker-compose.base.yml + service: druid-router-custom-check-tls + env_file: + - ${OVERRIDE_ENV} + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-metadata-storage + - druid-overlord + - druid-coordinator + - druid-historical + - druid-middlemanager + - druid-broker + - druid-router + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 \ No newline at end of file diff --git a/integration-tests/docker/docker-compose.yml b/integration-tests/docker/docker-compose.yml new file mode 100644 index 000000000000..43fe11545d3d --- /dev/null +++ b/integration-tests/docker/docker-compose.yml @@ -0,0 +1,173 @@ +# 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. + +version: "2.2" +services: + druid-zookeeper-kafka: + extends: + file: docker-compose.base.yml + service: druid-zookeeper-kafka + + druid-metadata-storage: + extends: + file: docker-compose.base.yml + service: druid-metadata-storage + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + depends_on: + - druid-zookeeper-kafka + + druid-overlord: + extends: + file: docker-compose.base.yml + service: druid-overlord + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-metadata-storage:druid-metadata-storage + - druid-zookeeper-kafka:druid-zookeeper-kafka + depends_on: + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-coordinator: + extends: + file: docker-compose.base.yml + service: druid-coordinator + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-overlord:druid-overlord + - druid-metadata-storage:druid-metadata-storage + - druid-zookeeper-kafka:druid-zookeeper-kafka + depends_on: + - druid-overlord + - druid-metadata-storage + - druid-zookeeper-kafka + + druid-historical: + extends: + file: docker-compose.base.yml + service: druid-historical + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + depends_on: + - druid-zookeeper-kafka + + druid-middlemanager: + extends: + file: docker-compose.base.yml + service: druid-middlemanager + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-overlord:druid-overlord + depends_on: + - druid-zookeeper-kafka + - druid-overlord + + druid-broker: + extends: + file: docker-compose.base.yml + service: druid-broker + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-middlemanager:druid-middlemanager + - druid-historical:druid-historical + depends_on: + - druid-zookeeper-kafka + - druid-middlemanager + - druid-historical + + druid-router: + extends: + file: docker-compose.base.yml + service: druid-router + environment: + - DRUID_INTEGRATION_TEST_GROUP=${DRUID_INTEGRATION_TEST_GROUP} + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-coordinator + - druid-broker + + druid-router-permissive-tls: + extends: + file: docker-compose.base.yml + service: druid-router-permissive-tls + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-metadata-storage + - druid-overlord + - druid-coordinator + - druid-historical + - druid-middlemanager + - druid-broker + - druid-router + + druid-router-no-client-auth-tls: + extends: + file: docker-compose.base.yml + service: druid-router-no-client-auth-tls + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-metadata-storage + - druid-overlord + - druid-coordinator + - druid-historical + - druid-middlemanager + - druid-broker + - druid-router + + druid-router-custom-check-tls: + extends: + file: docker-compose.base.yml + service: druid-router-custom-check-tls + links: + - druid-zookeeper-kafka:druid-zookeeper-kafka + - druid-coordinator:druid-coordinator + - druid-broker:druid-broker + depends_on: + - druid-zookeeper-kafka + - druid-metadata-storage + - druid-overlord + - druid-coordinator + - druid-historical + - druid-middlemanager + - druid-broker + - druid-router + +networks: + druid-it-net: + name: druid-it-net + ipam: + config: + - subnet: 172.172.172.0/24 \ No newline at end of file diff --git a/integration-tests/docker/environment-configs/override-examples/s3 b/integration-tests/docker/environment-configs/override-examples/s3 index cdca76490625..9146e985757f 100644 --- a/integration-tests/docker/environment-configs/override-examples/s3 +++ b/integration-tests/docker/environment-configs/override-examples/s3 @@ -27,4 +27,4 @@ druid_storage_baseKey=druid/segments druid_s3_accessKey= druid_s3_secretKey= AWS_REGION= -druid_extensions_loadList=["druid-s3-extensions"] \ No newline at end of file +druid_extensions_loadList=["druid-s3-extensions","druid-hdfs-storage"] \ No newline at end of file diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 9d10680dc957..5aa63025d27d 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -361,8 +361,8 @@ integration-tests false - false - false + false + false \ Россия\ 한국\ 中国!? @@ -374,21 +374,22 @@ exec-maven-plugin - build-and-start-druid-cluster + docker-package exec pre-integration-test - ${start.hadoop.docker} - ${skip.start.docker} - ${jvm.runtime} - ${groups} - ${override.config.path} - ${resource.file.dir.path}> + ${start.hadoop.docker} + ${jvm.runtime} + ${groups} + ${override.config.path} + ${resource.file.dir.path} + ${docker.build.skip} + ${docker.run.skip} - ${project.basedir}/run_cluster.sh + ${project.basedir}/build_run_cluster.sh @@ -399,13 +400,14 @@ post-integration-test - ${skip.stop.docker} + ${docker.run.skip} ${project.basedir}/stop_cluster.sh + org.apache.maven.plugins maven-failsafe-plugin @@ -446,6 +448,7 @@ + de.thetaphi forbiddenapis @@ -460,6 +463,7 @@ + int-tests-config-file diff --git a/integration-tests/run_cluster.sh b/integration-tests/run_cluster.sh deleted file mode 100755 index faaa4eaa1eed..000000000000 --- a/integration-tests/run_cluster.sh +++ /dev/null @@ -1,247 +0,0 @@ -#!/usr/bin/env bash -# 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. - -# Skip starting docker if flag set (For use during development) -if [ -n "$DRUID_INTEGRATION_TEST_SKIP_START_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_SKIP_START_DOCKER" == true ] - then - exit 0 - fi - -# Cleanup old images/containers -{ - for node in druid-historical druid-coordinator druid-overlord druid-router druid-router-permissive-tls druid-router-no-client-auth-tls druid-router-custom-check-tls druid-broker druid-middlemanager druid-zookeeper-kafka druid-metadata-storage druid-it-hadoop; - do - docker stop $node - docker rm $node - done - - docker network rm druid-it-net -} - -# Druid environment and jars setup -{ - # environment variables - DIR=$(cd $(dirname $0) && pwd) - HADOOP_DOCKER_DIR=$DIR/../examples/quickstart/tutorial/hadoop/docker - DOCKERDIR=$DIR/docker - SERVICE_SUPERVISORDS_DIR=$DOCKERDIR/service-supervisords - ENVIRONMENT_CONFIGS_DIR=$DOCKERDIR/environment-configs - SHARED_DIR=${HOME}/shared - SUPERVISORDIR=/usr/lib/druid/conf - RESOURCEDIR=$DIR/src/test/resources - - # so docker IP addr will be known during docker build - echo ${DOCKER_IP:=127.0.0.1} > $DOCKERDIR/docker_ip - - # setup client keystore - ./docker/tls/generate-client-certs-and-keystores.sh - rm -rf docker/client_tls - cp -r client_tls docker/client_tls - - # Make directories if they dont exist - mkdir -p $SHARED_DIR/hadoop_xml - mkdir -p $SHARED_DIR/hadoop-dependencies - mkdir -p $SHARED_DIR/logs - mkdir -p $SHARED_DIR/tasklogs - mkdir -p $SHARED_DIR/docker/extensions - mkdir -p $SHARED_DIR/docker/credentials - - # install druid jars - rm -rf $SHARED_DIR/docker - cp -R docker $SHARED_DIR/docker - mvn -B dependency:copy-dependencies -DoutputDirectory=$SHARED_DIR/docker/lib - - # move extensions into a seperate extension folder - # For druid-s3-extensions - mkdir -p $SHARED_DIR/docker/extensions/druid-s3-extensions - mv $SHARED_DIR/docker/lib/druid-s3-extensions-* $SHARED_DIR/docker/extensions/druid-s3-extensions - # For druid-azure-extensions - mkdir -p $SHARED_DIR/docker/extensions/druid-azure-extensions - mv $SHARED_DIR/docker/lib/druid-azure-extensions-* $SHARED_DIR/docker/extensions/druid-azure-extensions - # For druid-google-extensions - mkdir -p $SHARED_DIR/docker/extensions/druid-google-extensions - mv $SHARED_DIR/docker/lib/druid-google-extensions-* $SHARED_DIR/docker/extensions/druid-google-extensions - # For druid-hdfs-storage - mkdir -p $SHARED_DIR/docker/extensions/druid-hdfs-storage - mv $SHARED_DIR/docker/lib/druid-hdfs-storage-* $SHARED_DIR/docker/extensions/druid-hdfs-storage - # For druid-kinesis-indexing-service - mkdir -p $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service - mv $SHARED_DIR/docker/lib/druid-kinesis-indexing-service-* $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service - # For druid-parquet-extensions - mkdir -p $SHARED_DIR/docker/extensions/druid-parquet-extensions - mv $SHARED_DIR/docker/lib/druid-parquet-extensions-* $SHARED_DIR/docker/extensions/druid-parquet-extensions - # For druid-orc-extensions - mkdir -p $SHARED_DIR/docker/extensions/druid-orc-extensions - mv $SHARED_DIR/docker/lib/druid-orc-extensions-* $SHARED_DIR/docker/extensions/druid-orc-extensions - - # Pull Hadoop dependency if needed - if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ] - then - java -cp "$SHARED_DIR/docker/lib/*" -Ddruid.extensions.hadoopDependenciesDir="$SHARED_DIR/hadoop-dependencies" org.apache.druid.cli.Main tools pull-deps -h org.apache.hadoop:hadoop-client:2.8.5 -h org.apache.hadoop:hadoop-aws:2.8.5 -h org.apache.hadoop:hadoop-azure:2.8.5 - curl https://storage.googleapis.com/hadoop-lib/gcs/gcs-connector-hadoop2-latest.jar --output $SHARED_DIR/docker/lib/gcs-connector-hadoop2-latest.jar - fi - - # install logging config - cp src/main/resources/log4j2.xml $SHARED_DIR/docker/lib/log4j2.xml - - # copy the integration test jar, it provides test-only extension implementations - cp target/druid-integration-tests*.jar $SHARED_DIR/docker/lib - - # one of the integration tests needs the wikiticker sample data - mkdir -p $SHARED_DIR/wikiticker-it - cp ../examples/quickstart/tutorial/wikiticker-2015-09-12-sampled.json.gz $SHARED_DIR/wikiticker-it/wikiticker-2015-09-12-sampled.json.gz - cp docker/wiki-simple-lookup.json $SHARED_DIR/wikiticker-it/wiki-simple-lookup.json - - # copy other files if needed - if [ -n "$DRUID_INTEGRATION_TEST_RESOURCE_FILE_DIR_PATH" ] - then - cp -a $DRUID_INTEGRATION_TEST_RESOURCE_FILE_DIR_PATH/. $SHARED_DIR/docker/credentials/ - fi - - # setup all enviornment variables to be pass to the containers - COMMON_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/common -e DRUID_INTEGRATION_TEST_GROUP" - BROKER_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/broker" - COORDINATOR_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/coordinator" - HISTORICAL_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/historical" - MIDDLEMANAGER_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/middlemanager" - OVERLORD_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/overlord" - ROUTER_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/router" - ROUTER_CUSTOM_CHECK_TLS_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/router-custom-check-tls" - ROUTER_NO_CLIENT_AUTH_TLS_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/router-no-client-auth-tls" - ROUTER_PERMISSIVE_TLS_ENV="--env-file=$ENVIRONMENT_CONFIGS_DIR/router-permissive-tls" - - OVERRIDE_ENV="" - if [ -z "$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH" ] - then - echo "\$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH is not set. No override config file provided" - if [ "$DRUID_INTEGRATION_TEST_GROUP" = "s3-deep-storage" ] || \ - [ "$DRUID_INTEGRATION_TEST_GROUP" = "gcs-deep-storage" ] || \ - [ "$DRUID_INTEGRATION_TEST_GROUP" = "azure-deep-storage" ]; then - echo "Test group $DRUID_INTEGRATION_TEST_GROUP requires override config file. Stopping test..." - exit 1 - fi - else - echo "\$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH is set with value ${DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH}" - OVERRIDE_ENV="--env-file=$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH" - fi -} - -# Create docker network -{ - docker network create --subnet=172.172.172.0/24 druid-it-net -} - -# Build Druid Cluster Image -if [ -z "$DRUID_INTEGRATION_TEST_JVM_RUNTIME" ] -then - echo "\$DRUID_INTEGRATION_TEST_JVM_RUNTIME is not set. Running integration test with image running Java 8" - docker build -t druid/cluster --build-arg DOCKER_IMAGE=imply/druiditbase:openjdk-1.8.0_191-1 $SHARED_DIR/docker -else - echo "\$DRUID_INTEGRATION_TEST_JVM_RUNTIME is set with value ${DRUID_INTEGRATION_TEST_JVM_RUNTIME}" - case "${DRUID_INTEGRATION_TEST_JVM_RUNTIME}" in - 8) - echo "Running integration test with image running Java 8" - docker build -t druid/cluster --build-arg DOCKER_IMAGE=imply/druiditbase:openjdk-1.8.0_191-1 $SHARED_DIR/docker - ;; - 11) - echo "Running integration test with image running Java 11" - docker build -t druid/cluster --build-arg DOCKER_IMAGE=imply/druiditbase:openjdk-11.0.5-1 $SHARED_DIR/docker - ;; - *) - echo "Invalid JVM Runtime given. Stopping" - exit 1 - ;; - esac -fi - -# Build Hadoop docker if needed -if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ] -then - docker build -t druid-it/hadoop:2.8.5 $HADOOP_DOCKER_DIR -fi - - -# Start docker containers for all Druid processes and dependencies -{ - # Start Hadoop docker if needed - if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ] - then - # Start Hadoop docker container - docker run -d --privileged --net druid-it-net --ip 172.172.172.13 -h druid-it-hadoop --name druid-it-hadoop -p 2049:2049 -p 2122:2122 -p 8020:8020 -p 8021:8021 -p 8030:8030 -p 8031:8031 -p 8032:8032 -p 8033:8033 -p 8040:8040 -p 8042:8042 -p 8088:8088 -p 8443:8443 -p 9000:9000 -p 10020:10020 -p 19888:19888 -p 34455:34455 -p 49707:49707 -p 50010:50010 -p 50020:50020 -p 50030:50030 -p 50060:50060 -p 50070:50070 -p 50075:50075 -p 50090:50090 -p 51111:51111 -v $RESOURCEDIR:/resources -v $SHARED_DIR:/shared druid-it/hadoop:2.8.5 sh -c "/etc/bootstrap.sh && tail -f /dev/null" - - # wait for hadoop namenode to be up - echo "Waiting for hadoop namenode to be up" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid" - while [ $? -ne 0 ] - do - sleep 2 - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid" - done - echo "Finished waiting for Hadoop namenode" - - # Setup hadoop druid dirs - echo "Setting up druid hadoop dirs" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid/segments" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /quickstart" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod 777 /druid" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod 777 /druid/segments" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod 777 /quickstart" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod -R 777 /tmp" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod -R 777 /user" - # Copy data files to Hadoop container - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -put /shared/wikiticker-it/wikiticker-2015-09-12-sampled.json.gz /quickstart/wikiticker-2015-09-12-sampled.json.gz" - docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -put /resources/data/batch_index /batch_index" - echo "Finished setting up druid hadoop dirs" - - echo "Copying Hadoop XML files to shared" - docker exec -t druid-it-hadoop sh -c "cp /usr/local/hadoop/etc/hadoop/*.xml /shared/hadoop_xml" - echo "Copied Hadoop XML files to shared" - fi - - # Start zookeeper and kafka - docker run -d --privileged --net druid-it-net --ip 172.172.172.2 ${COMMON_ENV} --name druid-zookeeper-kafka -p 2181:2181 -p 9092:9092 -p 9093:9093 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/zookeeper.conf:$SUPERVISORDIR/zookeeper.conf -v $SERVICE_SUPERVISORDS_DIR/kafka.conf:$SUPERVISORDIR/kafka.conf druid/cluster - - # Start MYSQL - docker run -d --privileged --net druid-it-net --ip 172.172.172.3 ${COMMON_ENV} --name druid-metadata-storage -p 3306:3306 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/metadata-storage.conf:$SUPERVISORDIR/metadata-storage.conf druid/cluster - - # Start Overlord - docker run -d --privileged --net druid-it-net --ip 172.172.172.4 ${COMMON_ENV} ${OVERLORD_ENV} ${OVERRIDE_ENV} --name druid-overlord -p 5009:5009 -p 8090:8090 -p 8290:8290 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster - - # Start Coordinator - docker run -d --privileged --net druid-it-net --ip 172.172.172.5 ${COMMON_ENV} ${COORDINATOR_ENV} ${OVERRIDE_ENV} --name druid-coordinator -p 5006:5006 -p 8081:8081 -p 8281:8281 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-overlord:druid-overlord --link druid-metadata-storage:druid-metadata-storage --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster - - # Start Historical - docker run -d --privileged --net druid-it-net --ip 172.172.172.6 ${COMMON_ENV} ${HISTORICAL_ENV} ${OVERRIDE_ENV} --name druid-historical -p 5007:5007 -p 8083:8083 -p 8283:8283 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka druid/cluster - - # Start Middlemanger - docker run -d --privileged --net druid-it-net --ip 172.172.172.7 ${COMMON_ENV} ${MIDDLEMANAGER_ENV} ${OVERRIDE_ENV} --name druid-middlemanager -p 5008:5008 -p 8091:8091 -p 8291:8291 -p 8100:8100 -p 8101:8101 -p 8102:8102 -p 8103:8103 -p 8104:8104 -p 8105:8105 -p 8300:8300 -p 8301:8301 -p 8302:8302 -p 8303:8303 -p 8304:8304 -p 8305:8305 -v $RESOURCEDIR:/resources -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-overlord:druid-overlord druid/cluster - - # Start Broker - docker run -d --privileged --net druid-it-net --ip 172.172.172.8 ${COMMON_ENV} ${BROKER_ENV} ${OVERRIDE_ENV} --name druid-broker -p 5005:5005 -p 8082:8082 -p 8282:8282 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-middlemanager:druid-middlemanager --link druid-historical:druid-historical druid/cluster - - # Start Router - docker run -d --privileged --net druid-it-net --ip 172.172.172.9 ${COMMON_ENV} ${ROUTER_ENV} ${OVERRIDE_ENV} --name druid-router -p 8888:8888 -p 5004:5004 -p 9088:9088 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster - - # Start Router with permissive TLS settings (client auth enabled, no hostname verification, no revocation check) - docker run -d --privileged --net druid-it-net --ip 172.172.172.10 ${COMMON_ENV} ${ROUTER_PERMISSIVE_TLS_ENV} ${OVERRIDE_ENV} --name druid-router-permissive-tls -p 5001:5001 -p 8889:8889 -p 9089:9089 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster - - # Start Router with TLS but no client auth - docker run -d --privileged --net druid-it-net --ip 172.172.172.11 ${COMMON_ENV} ${ROUTER_NO_CLIENT_AUTH_TLS_ENV} ${OVERRIDE_ENV} --name druid-router-no-client-auth-tls -p 5002:5002 -p 8890:8890 -p 9090:9090 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster - - # Start Router with custom TLS cert checkers - docker run -d --privileged --net druid-it-net --ip 172.172.172.12 ${COMMON_ENV} ${ROUTER_CUSTOM_CHECK_TLS_ENV} ${OVERRIDE_ENV} --hostname druid-router-custom-check-tls --name druid-router-custom-check-tls -p 5003:5003 -p 8891:8891 -p 9091:9091 -v $SHARED_DIR:/shared -v $SERVICE_SUPERVISORDS_DIR/druid.conf:$SUPERVISORDIR/druid.conf --link druid-zookeeper-kafka:druid-zookeeper-kafka --link druid-coordinator:druid-coordinator --link druid-broker:druid-broker druid/cluster - } \ No newline at end of file diff --git a/integration-tests/script/copy_hadoop_resources.sh b/integration-tests/script/copy_hadoop_resources.sh new file mode 100755 index 000000000000..82dd0d023d59 --- /dev/null +++ b/integration-tests/script/copy_hadoop_resources.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# 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. + +# wait for hadoop namenode to be up +echo "Waiting for hadoop namenode to be up" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid" +while [ $? -ne 0 ] +do + sleep 2 + docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid" +done +echo "Finished waiting for Hadoop namenode" + +# Setup hadoop druid dirs +echo "Setting up druid hadoop dirs" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /druid/segments" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -mkdir -p /quickstart" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod 777 /druid" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod 777 /druid/segments" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod 777 /quickstart" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod -R 777 /tmp" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -chmod -R 777 /user" +# Copy data files to Hadoop container +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -put /shared/wikiticker-it/wikiticker-2015-09-12-sampled.json.gz /quickstart/wikiticker-2015-09-12-sampled.json.gz" +docker exec -t druid-it-hadoop sh -c "./usr/local/hadoop/bin/hdfs dfs -put /resources/data/batch_index /batch_index" +echo "Finished setting up druid hadoop dirs" + +echo "Copying Hadoop XML files to shared" +docker exec -t druid-it-hadoop sh -c "cp /usr/local/hadoop/etc/hadoop/*.xml /shared/hadoop_xml" +echo "Copied Hadoop XML files to shared" \ No newline at end of file diff --git a/integration-tests/script/copy_resources.sh b/integration-tests/script/copy_resources.sh new file mode 100755 index 000000000000..eb3a1b594d64 --- /dev/null +++ b/integration-tests/script/copy_resources.sh @@ -0,0 +1,80 @@ +#!/usr/bin/env bash +# 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. + +# setup client keystore +./docker/tls/generate-client-certs-and-keystores.sh +rm -rf docker/client_tls +cp -r client_tls docker/client_tls + +# Make directories if they dont exist +mkdir -p $SHARED_DIR/hadoop_xml +mkdir -p $SHARED_DIR/hadoop-dependencies +mkdir -p $SHARED_DIR/logs +mkdir -p $SHARED_DIR/tasklogs +mkdir -p $SHARED_DIR/docker/extensions +mkdir -p $SHARED_DIR/docker/credentials + +# install druid jars +rm -rf $SHARED_DIR/docker +cp -R docker $SHARED_DIR/docker +mvn -B dependency:copy-dependencies -DoutputDirectory=$SHARED_DIR/docker/lib + +# move extensions into a seperate extension folder +# For druid-s3-extensions +mkdir -p $SHARED_DIR/docker/extensions/druid-s3-extensions +mv $SHARED_DIR/docker/lib/druid-s3-extensions-* $SHARED_DIR/docker/extensions/druid-s3-extensions +# For druid-azure-extensions +mkdir -p $SHARED_DIR/docker/extensions/druid-azure-extensions +mv $SHARED_DIR/docker/lib/druid-azure-extensions-* $SHARED_DIR/docker/extensions/druid-azure-extensions +# For druid-google-extensions +mkdir -p $SHARED_DIR/docker/extensions/druid-google-extensions +mv $SHARED_DIR/docker/lib/druid-google-extensions-* $SHARED_DIR/docker/extensions/druid-google-extensions +# For druid-hdfs-storage +mkdir -p $SHARED_DIR/docker/extensions/druid-hdfs-storage +mv $SHARED_DIR/docker/lib/druid-hdfs-storage-* $SHARED_DIR/docker/extensions/druid-hdfs-storage +# For druid-kinesis-indexing-service +mkdir -p $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service +mv $SHARED_DIR/docker/lib/druid-kinesis-indexing-service-* $SHARED_DIR/docker/extensions/druid-kinesis-indexing-service +# For druid-parquet-extensions +mkdir -p $SHARED_DIR/docker/extensions/druid-parquet-extensions +mv $SHARED_DIR/docker/lib/druid-parquet-extensions-* $SHARED_DIR/docker/extensions/druid-parquet-extensions +# For druid-orc-extensions +mkdir -p $SHARED_DIR/docker/extensions/druid-orc-extensions +mv $SHARED_DIR/docker/lib/druid-orc-extensions-* $SHARED_DIR/docker/extensions/druid-orc-extensions + +# Pull Hadoop dependency if needed +if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ] +then + java -cp "$SHARED_DIR/docker/lib/*" -Ddruid.extensions.hadoopDependenciesDir="$SHARED_DIR/hadoop-dependencies" org.apache.druid.cli.Main tools pull-deps -h org.apache.hadoop:hadoop-client:2.8.5 -h org.apache.hadoop:hadoop-aws:2.8.5 -h org.apache.hadoop:hadoop-azure:2.8.5 + curl https://storage.googleapis.com/hadoop-lib/gcs/gcs-connector-hadoop2-latest.jar --output $SHARED_DIR/docker/lib/gcs-connector-hadoop2-latest.jar +fi + +# install logging config +cp src/main/resources/log4j2.xml $SHARED_DIR/docker/lib/log4j2.xml + +# copy the integration test jar, it provides test-only extension implementations +cp target/druid-integration-tests*.jar $SHARED_DIR/docker/lib + +# one of the integration tests needs the wikiticker sample data +mkdir -p $SHARED_DIR/wikiticker-it +cp ../examples/quickstart/tutorial/wikiticker-2015-09-12-sampled.json.gz $SHARED_DIR/wikiticker-it/wikiticker-2015-09-12-sampled.json.gz +cp docker/wiki-simple-lookup.json $SHARED_DIR/wikiticker-it/wiki-simple-lookup.json + +# copy other files if needed +if [ -n "$DRUID_INTEGRATION_TEST_RESOURCE_FILE_DIR_PATH" ] +then + cp -a $DRUID_INTEGRATION_TEST_RESOURCE_FILE_DIR_PATH/. $SHARED_DIR/docker/credentials/ +fi \ No newline at end of file diff --git a/integration-tests/script/docker_build_containers.sh b/integration-tests/script/docker_build_containers.sh new file mode 100755 index 000000000000..30aa56a8de8f --- /dev/null +++ b/integration-tests/script/docker_build_containers.sh @@ -0,0 +1,44 @@ +#!/usr/bin/env bash +# 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. + +# Build Druid Cluster Image +if [ -z "$DRUID_INTEGRATION_TEST_JVM_RUNTIME" ] +then + echo "\$DRUID_INTEGRATION_TEST_JVM_RUNTIME is not set. Build druid-cluster with Java 8" + docker build -t druid/cluster --build-arg DOCKER_IMAGE=imply/druiditbase:openjdk-1.8.0_191-1 $SHARED_DIR/docker +else + echo "\$DRUID_INTEGRATION_TEST_JVM_RUNTIME is set with value ${DRUID_INTEGRATION_TEST_JVM_RUNTIME}" + case "${DRUID_INTEGRATION_TEST_JVM_RUNTIME}" in + 8) + echo "Build druid-cluster with Java 8" + docker build -t druid/cluster --build-arg DOCKER_IMAGE=imply/druiditbase:openjdk-1.8.0_191-1 $SHARED_DIR/docker + ;; + 11) + echo "Build druid-cluster with Java 11" + docker build -t druid/cluster --build-arg DOCKER_IMAGE=imply/druiditbase:openjdk-11.0.5-1 $SHARED_DIR/docker + ;; + *) + echo "Invalid JVM Runtime given. Stopping" + exit 1 + ;; + esac +fi + +# Build Hadoop docker if needed +if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ] +then + docker build -t druid-it/hadoop:2.8.5 $HADOOP_DOCKER_DIR +fi diff --git a/integration-tests/script/docker_run_cluster.sh b/integration-tests/script/docker_run_cluster.sh new file mode 100755 index 000000000000..48fcd02c923e --- /dev/null +++ b/integration-tests/script/docker_run_cluster.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# 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. + +# Create docker network +{ + docker network create --subnet=172.172.172.0/24 druid-it-net +} + +if [ -z "$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH" ] +then + echo "\$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH is not set. No override config file provided" + if [ "$DRUID_INTEGRATION_TEST_GROUP" = "s3-deep-storage" ] || \ + [ "$DRUID_INTEGRATION_TEST_GROUP" = "gcs-deep-storage" ] || \ + [ "$DRUID_INTEGRATION_TEST_GROUP" = "azure-deep-storage" ] || \ + [ "$DRUID_INTEGRATION_TEST_GROUP" = "hdfs-deep-storage" ] || \ + [ "$DRUID_INTEGRATION_TEST_GROUP" = "s3-ingestion" ] || \ + [ "$DRUID_INTEGRATION_TEST_GROUP" = "kinesis-index" ] || \ + [ "$DRUID_INTEGRATION_TEST_GROUP" = "kinesis-data-format" ]; then + echo "Test group $DRUID_INTEGRATION_TEST_GROUP requires override config file. Stopping test..." + exit 1 + fi +else + echo "\$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH is set with value ${DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH}" +fi + +# Start docker containers for all Druid processes and dependencies +{ + # Start Hadoop docker if needed + if [ -n "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_START_HADOOP_DOCKER" == true ] + then + # Start Hadoop docker container + docker-compose -f ${DOCKERDIR}/docker-compose.druid-hadoop.yml up -d + fi + + if [ -z "$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH" ] + then + docker-compose -f ${DOCKERDIR}/docker-compose.yml up -d + else + # run druid cluster with override config + OVERRIDE_ENV=$DRUID_INTEGRATION_TEST_OVERRIDE_CONFIG_PATH docker-compose -f ${DOCKERDIR}/docker-compose.override-env.yml up -d + fi +} diff --git a/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java b/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java index 6166359693a1..5d2132322290 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/guice/ITTLSCertificateCheckerModule.java @@ -46,7 +46,7 @@ public void configure(Binder binder) @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java index 7309e7c8641f..5c64dcd626b0 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/AbstractITBatchIndexTest.java @@ -25,7 +25,6 @@ import org.apache.druid.indexing.common.task.batch.parallel.PartialDimensionDistributionTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialGenericSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask; -import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentMergeTask; import org.apache.druid.indexing.common.task.batch.parallel.PartialRangeSegmentGenerateTask; import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseSubTask; import org.apache.druid.java.util.common.ISE; @@ -312,7 +311,6 @@ private long countCompleteSubTasks(final String dataSource, final boolean perfec return t.getType().equals(SinglePhaseSubTask.TYPE); } else { return t.getType().equalsIgnoreCase(PartialHashSegmentGenerateTask.TYPE) - || t.getType().equalsIgnoreCase(PartialHashSegmentMergeTask.TYPE) || t.getType().equalsIgnoreCase(PartialDimensionDistributionTask.TYPE) || t.getType().equalsIgnoreCase(PartialRangeSegmentGenerateTask.TYPE) || t.getType().equalsIgnoreCase(PartialGenericSegmentMergeTask.TYPE); diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java index 7d34b47d7065..269c74d4cd19 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITTestCoordinatorPausedTest.java @@ -34,7 +34,7 @@ @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITTestCoordinatorPausedTest extends AbstractITBatchIndexTest { - private static final Logger LOG = new Logger(ITUnionQueryTest.class); + private static final Logger LOG = new Logger(ITTestCoordinatorPausedTest.class); private static final String INDEX_DATASOURCE = "wikipedia_index_test"; private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json"; private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json"; diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITUnionQueryTest.java b/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java similarity index 96% rename from integration-tests/src/test/java/org/apache/druid/tests/indexer/ITUnionQueryTest.java rename to integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java index 4bf68ad8834c..c720b1fab5be 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITUnionQueryTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/query/ITUnionQueryTest.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.tests.indexer; +package org.apache.druid.tests.query; import com.google.inject.Inject; import org.apache.commons.io.IOUtils; @@ -39,6 +39,8 @@ import org.apache.druid.testing.utils.ITRetryUtil; import org.apache.druid.testing.utils.ServerDiscoveryUtil; import org.apache.druid.tests.TestNGGroup; +import org.apache.druid.tests.indexer.AbstractITBatchIndexTest; +import org.apache.druid.tests.indexer.AbstractIndexerTest; import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.DateTime; @@ -62,7 +64,7 @@ public class ITUnionQueryTest extends AbstractIndexerTest private static final String UNION_TASK_RESOURCE = "/indexer/wikipedia_union_index_task.json"; private static final String EVENT_RECEIVER_SERVICE_PREFIX = "eventReceiverServiceName"; private static final String UNION_DATA_FILE = "/data/union_query/wikipedia_index_data.json"; - private static final String UNION_QUERIES_RESOURCE = "/indexer/union_queries.json"; + private static final String UNION_QUERIES_RESOURCE = "/queries/union_queries.json"; private static final String UNION_DATASOURCE = "wikipedia_index_test"; @Inject @@ -92,7 +94,7 @@ public void testUnionQuery() throws IOException closer.register(unloader(fullDatasourceName + i)); } try { - // Load 4 datasources with same dimensions + // Load 3 datasources with same dimensions String task = setShutOffTime( getResourceAsString(UNION_TASK_RESOURCE), DateTimes.utc(System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(3)) @@ -117,6 +119,7 @@ public void testUnionQuery() throws IOException () -> { for (int i = 0; i < numTasks; i++) { final int countRows = queryHelper.countRows(fullDatasourceName + i, "2013-08-31/2013-09-01"); + // there are 10 rows, but query only covers the first 5 if (countRows < 5) { LOG.warn("%d events have been ingested to %s so far", countRows, fullDatasourceName + i); return false; diff --git a/integration-tests/src/test/resources/indexer/union_queries.json b/integration-tests/src/test/resources/queries/union_queries.json similarity index 100% rename from integration-tests/src/test/resources/indexer/union_queries.json rename to integration-tests/src/test/resources/queries/union_queries.json diff --git a/integration-tests/stop_cluster.sh b/integration-tests/stop_cluster.sh index 2828a0ff8a96..d75e73faca5e 100755 --- a/integration-tests/stop_cluster.sh +++ b/integration-tests/stop_cluster.sh @@ -15,16 +15,15 @@ # limitations under the License. # Skip stopping docker if flag set (For use during development) -if [ -n "$DRUID_INTEGRATION_TEST_SKIP_STOP_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_SKIP_STOP_DOCKER" == true ] +if [ -n "$DRUID_INTEGRATION_TEST_SKIP_RUN_DOCKER" ] && [ "$DRUID_INTEGRATION_TEST_SKIP_RUN_DOCKER" == true ] then exit 0 fi for node in druid-historical druid-coordinator druid-overlord druid-router druid-router-permissive-tls druid-router-no-client-auth-tls druid-router-custom-check-tls druid-broker druid-middlemanager druid-zookeeper-kafka druid-metadata-storage druid-it-hadoop; - do -docker stop $node -docker rm $node + docker stop $node + docker rm $node done docker network rm druid-it-net diff --git a/licenses.yaml b/licenses.yaml index 6680ade81495..24d93ee0d1c3 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -1975,7 +1975,7 @@ name: LZ4 Java license_category: binary module: java-core license_name: Apache License version 2.0 -version: 1.6.0 +version: 1.7.1 libraries: - org.lz4: lz4-java @@ -3251,7 +3251,7 @@ libraries: --- name: Apache Kafka -version: 2.2.2 +version: 2.5.0 license_category: binary module: extensions/druid-kafka-indexing-service license_name: Apache License version 2.0 @@ -3625,7 +3625,7 @@ name: PostgreSQL JDBC Driver license_category: binary module: extensions/druid-lookups-cached-single license_name: BSD-2-Clause License -version: 42.2.8 +version: 42.2.14 copyright: PostgreSQL Global Development Group license_file_path: licenses/bin/postgresql.BSD2 libraries: @@ -3637,7 +3637,7 @@ name: PostgreSQL JDBC Driver license_category: binary module: extensions/druid-lookups-cached-global license_name: BSD-2-Clause License -version: 42.2.8 +version: 42.2.14 copyright: PostgreSQL Global Development Group license_file_path: licenses/bin/postgresql.BSD2 libraries: @@ -3649,7 +3649,7 @@ name: PostgreSQL JDBC Driver license_category: binary module: extensions/postgresql-metadata-storage license_name: BSD-2-Clause License -version: 42.2.8 +version: 42.2.14 copyright: PostgreSQL Global Development Group license_file_path: licenses/bin/postgresql.BSD2 libraries: @@ -4159,7 +4159,7 @@ name: Apache Kafka license_category: binary module: extensions/kafka-extraction-namespace license_name: Apache License version 2.0 -version: 2.2.2 +version: 2.5.0 libraries: - org.apache.kafka: kafka_2.12 - org.apache.kafka: kafka-clients diff --git a/pom.xml b/pom.xml index aeffceb8a5cb..c6e191a06a70 100644 --- a/pom.xml +++ b/pom.xml @@ -78,7 +78,7 @@ 0.9.0.M2 4.3.0 2.12.0 - 2.2.2 + 2.5.0 2.0.0 2.2.4 1.15.0 @@ -103,7 +103,7 @@ 4.1.48.Final v10.14.2 6.5.0 - 42.2.8 + 42.2.14 3.11.0 1.7.12 @@ -111,6 +111,9 @@ 2.0.2 1.11.199 2.8.0 + 3.4.14 2.5.7 @@ -189,6 +192,7 @@ extensions-contrib/tdigestsketch extensions-contrib/influxdb-emitter extensions-contrib/gce-extensions + extensions-contrib/prometheus-emitter distribution @@ -771,7 +775,7 @@ org.lz4 lz4-java - 1.6.0 + 1.7.1 com.google.protobuf diff --git a/processing/pom.xml b/processing/pom.xml index d3b244f3c2b1..5355bf1a16a7 100644 --- a/processing/pom.xml +++ b/processing/pom.xml @@ -91,6 +91,10 @@ commons-io commons-io + + org.apache.commons + commons-math3 + commons-net commons-net diff --git a/processing/src/main/java/org/apache/druid/query/BaseQuery.java b/processing/src/main/java/org/apache/druid/query/BaseQuery.java index f95961cf40a6..68be7ba48fd0 100644 --- a/processing/src/main/java/org/apache/druid/query/BaseQuery.java +++ b/processing/src/main/java/org/apache/druid/query/BaseQuery.java @@ -123,7 +123,7 @@ public static QuerySegmentSpec getQuerySegmentSpecForLookUp(BaseQuery query) { return DataSourceAnalysis.forDataSource(query.getDataSource()) .getBaseQuerySegmentSpec() - .orElse(query.getQuerySegmentSpec()); + .orElseGet(query::getQuerySegmentSpec); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/DataSource.java b/processing/src/main/java/org/apache/druid/query/DataSource.java index 549b06b0b3eb..d12a8eeabab8 100644 --- a/processing/src/main/java/org/apache/druid/query/DataSource.java +++ b/processing/src/main/java/org/apache/druid/query/DataSource.java @@ -28,14 +28,15 @@ /** * Represents a source... of data... for a query. Analogous to the "FROM" clause in SQL. */ -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = LegacyDataSource.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = TableDataSource.class) @JsonSubTypes({ @JsonSubTypes.Type(value = TableDataSource.class, name = "table"), @JsonSubTypes.Type(value = QueryDataSource.class, name = "query"), @JsonSubTypes.Type(value = UnionDataSource.class, name = "union"), @JsonSubTypes.Type(value = JoinDataSource.class, name = "join"), @JsonSubTypes.Type(value = LookupDataSource.class, name = "lookup"), - @JsonSubTypes.Type(value = InlineDataSource.class, name = "inline") + @JsonSubTypes.Type(value = InlineDataSource.class, name = "inline"), + @JsonSubTypes.Type(value = GlobalTableDataSource.class, name = "globalTable") }) public interface DataSource { @@ -70,6 +71,15 @@ public interface DataSource /** * Returns true if all servers have a full copy of this datasource. True for things like inline, lookup, etc, or * for queries of those. + * + * Currently this is coupled with joinability - if this returns true then the query engine expects there exists a + * {@link org.apache.druid.segment.join.JoinableFactory} which might build a + * {@link org.apache.druid.segment.join.Joinable} for this datasource directly. If a subquery 'inline' join is + * required to join this datasource on the right hand side, then this value must be false for now. + * + * In the future, instead of directly using this method, the query planner and engine should consider + * {@link org.apache.druid.segment.join.JoinableFactory#isDirectlyJoinable(DataSource)} when determining if the + * right hand side is directly joinable, which would allow decoupling this property from joins. */ boolean isGlobal(); diff --git a/processing/src/main/java/org/apache/druid/query/GlobalTableDataSource.java b/processing/src/main/java/org/apache/druid/query/GlobalTableDataSource.java new file mode 100644 index 000000000000..da5f1390ca3d --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/GlobalTableDataSource.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.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; + +/** + * {@link TableDataSource} variant for globally available 'broadcast' segments. If bound to a + * {@link org.apache.druid.segment.join.JoinableFactory} that can create an + * {@link org.apache.druid.segment.join.table.IndexedTable} using DruidBinders.joinableFactoryBinder, this allows + * optimal usage of segments using this DataSource type in join operations (because they are global), and so can be + * pushed down to historicals as a {@link JoinDataSource}, instead of requiring a subquery join using + * {@link InlineDataSource} to construct an {@link org.apache.druid.segment.join.table.IndexedTable} on the fly on the + * broker. Because it is also a {@link TableDataSource}, when queried directly, or on the left hand side of a join, + * they will be treated as any normal table datasource. + */ +@JsonTypeName("globalTable") +public class GlobalTableDataSource extends TableDataSource +{ + @JsonCreator + public GlobalTableDataSource(@JsonProperty("name") String name) + { + super(name); + } + + @Override + public boolean isGlobal() + { + return true; + } + + @Override + public String toString() + { + return "GlobalTableDataSource{" + + "name='" + getName() + '\'' + + '}'; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java index 509738ee6803..ee400f814fb8 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/QueryToolChest.java @@ -179,11 +179,10 @@ public abstract Function makePreComputeManipulatorFn( ); /** - * Generally speaking this is the exact same thing as makePreComputeManipulatorFn. It is leveraged in - * order to compute PostAggregators on results after they have been completely merged together, which - * should actually be done in the mergeResults() call instead of here. - *

- * This should never actually be overridden and it should be removed as quickly as possible. + * Generally speaking this is the exact same thing as makePreComputeManipulatorFn. It is leveraged in order to + * compute PostAggregators on results after they have been completely merged together. To minimize walks of segments, + * it is recommended to use mergeResults() call instead of this method if possible. However, this may not always be + * possible as we don’t always want to run PostAggregators and other stuff that happens there when you mergeResults. * * @param query The Query that is currently being processed * @param fn The function that should be applied to all metrics in the results diff --git a/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java b/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.java new file mode 100644 index 000000000000..41126dcaf4f0 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QueryUnsupportedException.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; + + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; +import java.net.InetAddress; + +/** + * This exception is for the query engine to surface when a query cannot be run. This can be due to the + * following reasons: 1) The query is not supported yet. 2) The query is not something Druid would ever supports. + * For these cases, the exact causes and details should also be documented in Druid user facing documents. + * + * As a {@link QueryException} it is expected to be serialied to a json response, but will be mapped to + * {@link #STATUS_CODE} instead of the default HTTP 500 status. + */ +public class QueryUnsupportedException extends QueryException +{ + private static final String ERROR_CLASS = QueryUnsupportedException.class.getName(); + public static final String ERROR_CODE = "Unsupported query"; + public static final int STATUS_CODE = 400; + + @JsonCreator + public QueryUnsupportedException( + @JsonProperty("error") @Nullable String errorCode, + @JsonProperty("errorMessage") String errorMessage, + @JsonProperty("errorClass") @Nullable String errorClass, + @JsonProperty("host") @Nullable String host + ) + { + super(errorCode, errorMessage, errorClass, host); + } + + public QueryUnsupportedException(String errorMessage) + { + super(ERROR_CODE, errorMessage, ERROR_CLASS, resolveHostname()); + } + + private static String resolveHostname() + { + String host; + try { + host = InetAddress.getLocalHost().getCanonicalHostName(); + } + catch (Exception e) { + host = null; + } + return host; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java index 8691441ec201..a92bbd6421b7 100644 --- a/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/ReferenceCountingSegmentQueryRunner.java @@ -22,50 +22,42 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.segment.ReferenceCounter; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; public class ReferenceCountingSegmentQueryRunner implements QueryRunner { private final QueryRunnerFactory> factory; - private final Segment segment; - private final ReferenceCounter segmentReferenceCounter; + private final SegmentReference segment; private final SegmentDescriptor descriptor; public ReferenceCountingSegmentQueryRunner( QueryRunnerFactory> factory, - Segment segment, - ReferenceCounter segmentReferenceCounter, + SegmentReference segment, SegmentDescriptor descriptor ) { this.factory = factory; this.segment = segment; - this.segmentReferenceCounter = segmentReferenceCounter; this.descriptor = descriptor; } @Override public Sequence run(final QueryPlus queryPlus, ResponseContext responseContext) { - if (segmentReferenceCounter.increment()) { + return segment.acquireReferences().map(closeable -> { try { final Sequence baseSequence = factory.createRunner(segment).run(queryPlus, responseContext); - - return Sequences.withBaggage(baseSequence, segmentReferenceCounter.decrementOnceCloseable()); + return Sequences.withBaggage(baseSequence, closeable); } catch (Throwable t) { try { - segmentReferenceCounter.decrement(); + closeable.close(); } catch (Exception e) { t.addSuppressed(e); } throw t; } - } else { - // Segment was closed before we had a chance to increment the reference count - return new ReportTimelineMissingSegmentQueryRunner(descriptor).run(queryPlus, responseContext); - } + }).orElseGet(() -> new ReportTimelineMissingSegmentQueryRunner(descriptor).run(queryPlus, responseContext)); } } diff --git a/processing/src/main/java/org/apache/druid/query/TableDataSource.java b/processing/src/main/java/org/apache/druid/query/TableDataSource.java index 4c371cf84510..469d5be21719 100644 --- a/processing/src/main/java/org/apache/druid/query/TableDataSource.java +++ b/processing/src/main/java/org/apache/druid/query/TableDataSource.java @@ -27,6 +27,7 @@ import java.util.Collections; import java.util.List; +import java.util.Objects; import java.util.Set; @JsonTypeName("table") @@ -40,6 +41,12 @@ public TableDataSource(@JsonProperty("name") String name) this.name = Preconditions.checkNotNull(name, "'name' must be nonnull"); } + @JsonCreator + public static TableDataSource create(final String name) + { + return new TableDataSource(name); + } + @JsonProperty public String getName() { @@ -93,27 +100,21 @@ public String toString() } @Override - public final boolean equals(Object o) + public boolean equals(Object o) { if (this == o) { return true; } - if (!(o instanceof TableDataSource)) { + if (o == null || getClass() != o.getClass()) { return false; } - TableDataSource that = (TableDataSource) o; - - if (!name.equals(that.name)) { - return false; - } - - return true; + return name.equals(that.name); } @Override - public final int hashCode() + public int hashCode() { - return name.hashCode(); + return Objects.hash(name); } } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java index 93f221771208..41e55a303fa7 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/AggregatorFactory.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.PerSegmentQueryOptimizationContext; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -68,7 +69,7 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory selectorFact /** * Returns whether or not this aggregation class supports vectorization. The default implementation returns false. */ - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { return false; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java index 599a2c492020..b7e9fcb14a77 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/CountAggregatorFactory.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -72,7 +73,7 @@ public Comparator getComparator() } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { return true; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java index 00bc89bfc26e..0e5867cf6f00 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/DoubleSumAggregatorFactory.java @@ -25,6 +25,9 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.segment.BaseDoubleColumnValueSelector; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -79,8 +82,12 @@ protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnS } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { + if (fieldName != null) { + final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); + return expression == null && (capabilities == null || ValueType.isNumeric(capabilities.getType())); + } return expression == null; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java index f76cd3d51516..f00710bb78ec 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregatorFactory.java @@ -29,6 +29,7 @@ import org.apache.druid.query.filter.IntervalDimFilter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -98,7 +99,7 @@ public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSelectorFa @Override public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory) { - Preconditions.checkState(canVectorize(), "Cannot vectorize"); + Preconditions.checkState(canVectorize(columnSelectorFactory), "Cannot vectorize"); final VectorValueMatcher valueMatcher = filter.makeVectorMatcher(columnSelectorFactory); return new FilteredVectorAggregator( valueMatcher, @@ -107,9 +108,9 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelect } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { - return delegate.canVectorize() && filter.canVectorizeMatcher(); + return delegate.canVectorize(columnInspector) && filter.canVectorizeMatcher(); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java index d9ed43ab22cf..0c61920fc28d 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FloatSumAggregatorFactory.java @@ -25,6 +25,8 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.segment.BaseFloatColumnValueSelector; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -78,12 +80,15 @@ protected VectorValueSelector vectorSelector(VectorColumnSelectorFactory columnS } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { + if (fieldName != null) { + final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); + return expression == null && (capabilities == null || capabilities.getType().isNumeric()); + } return expression == null; } - @Override protected VectorAggregator factorizeVector( VectorColumnSelectorFactory columnSelectorFactory, diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java index e7945a507cc1..337ce1817b63 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/LongSumAggregatorFactory.java @@ -25,6 +25,8 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorValueSelector; @@ -87,8 +89,12 @@ protected VectorAggregator factorizeVector( } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { + if (fieldName != null) { + final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); + return expression == null && (capabilities == null || capabilities.getType().isNumeric()); + } return expression == null; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java index d9d66e37599a..615f0b57db1a 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/NullableNumericAggregatorFactory.java @@ -65,7 +65,7 @@ public final BufferAggregator factorizeBuffered(ColumnSelectorFactory columnSele @Override public final VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelectorFactory) { - Preconditions.checkState(canVectorize(), "Cannot vectorize"); + Preconditions.checkState(canVectorize(columnSelectorFactory), "Cannot vectorize"); VectorValueSelector selector = vectorSelector(columnSelectorFactory); VectorAggregator aggregator = factorizeVector(columnSelectorFactory, selector); return NullHandling.replaceWithDefault() ? aggregator : new NullableNumericVectorAggregator(aggregator, selector); @@ -135,12 +135,11 @@ protected abstract BufferAggregator factorizeBuffered( * @see BufferAggregator */ protected VectorAggregator factorizeVector( - // Not used by current aggregators, but here for parity with "factorizeBuffered". - @SuppressWarnings("unused") VectorColumnSelectorFactory columnSelectorFactory, + VectorColumnSelectorFactory columnSelectorFactory, VectorValueSelector selector ) { - if (!canVectorize()) { + if (!canVectorize(columnSelectorFactory)) { throw new UnsupportedOperationException("Cannot vectorize"); } else { throw new UnsupportedOperationException("canVectorize returned true but 'factorizeVector' is not implemented"); diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java index 35e7938c8b5c..6468b6771976 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/SuppressedAggregatorFactory.java @@ -22,6 +22,7 @@ import org.apache.druid.query.PerSegmentQueryOptimizationContext; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; @@ -70,9 +71,9 @@ public VectorAggregator factorizeVector(VectorColumnSelectorFactory columnSelect } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { - return delegate.canVectorize(); + return delegate.canVectorize(columnInspector); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java index b3ab8c3b4ef8..e0b7686ed1f5 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactory.java @@ -38,6 +38,7 @@ import org.apache.druid.query.aggregation.cardinality.HyperLogLogCollectorAggregateCombiner; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.NilColumnValueSelector; import org.apache.druid.segment.column.ColumnCapabilities; @@ -57,17 +58,13 @@ public class HyperUniquesAggregatorFactory extends AggregatorFactory { public static Object estimateCardinality(@Nullable Object object, boolean round) { - if (object == null) { - return 0; - } - final HyperLogLogCollector collector = (HyperLogLogCollector) object; - // Avoid ternary, it causes estimateCardinalityRound to be cast to double. + // Avoid ternary for round check as it causes estimateCardinalityRound to be cast to double. if (round) { - return collector.estimateCardinalityRound(); + return collector == null ? 0L : collector.estimateCardinalityRound(); } else { - return collector.estimateCardinality(); + return collector == null ? 0d : collector.estimateCardinality(); } } @@ -140,7 +137,7 @@ public VectorAggregator factorizeVector(final VectorColumnSelectorFactory select } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { return true; } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java index d1da4d3189f0..782a2aad71a0 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregatorFactory.java @@ -31,7 +31,9 @@ import org.apache.druid.query.aggregation.BufferAggregator; import org.apache.druid.query.aggregation.VectorAggregator; import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -106,9 +108,10 @@ public VectorAggregator factorizeVector(final VectorColumnSelectorFactory select } @Override - public boolean canVectorize() + public boolean canVectorize(ColumnInspector columnInspector) { - return true; + final ColumnCapabilities capabilities = columnInspector.getColumnCapabilities(fieldName); + return capabilities == null || capabilities.getType().isNumeric(); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java index 75fe2740057c..23ef6847ec06 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java +++ b/processing/src/main/java/org/apache/druid/query/expression/ExprUtils.java @@ -98,4 +98,31 @@ static void checkLiteralArgument(String functionName, Expr arg, String argName) { Preconditions.checkArgument(arg.isLiteral(), createErrMsg(functionName, argName + " arg must be a literal")); } + + /** + * True if Expr is a string literal. + * + * In non-SQL-compliant null handling mode, this method will return true for null literals as well (because they are + * treated equivalently to empty strings, and we cannot tell the difference.) + * + * In SQL-compliant null handling mode, this method will return true for actual strings only, not nulls. + */ + static boolean isStringLiteral(final Expr expr) + { + return (expr.isLiteral() && expr.getLiteralValue() instanceof String) + || (NullHandling.replaceWithDefault() && isNullLiteral(expr)); + } + + /** + * True if Expr is a null literal. + * + * In non-SQL-compliant null handling mode, this method will return true for either a null literal or an empty string + * literal (they are treated equivalently and we cannot tell the difference). + * + * In SQL-compliant null handling mode, this method will only return true for an actual null literal. + */ + static boolean isNullLiteral(final Expr expr) + { + return expr.isLiteral() && expr.getLiteralValue() == null; + } } diff --git a/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java index 7428ede84647..9bef704a663e 100644 --- a/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java +++ b/processing/src/main/java/org/apache/druid/query/expression/RegexpExtractExprMacro.java @@ -52,12 +52,18 @@ public Expr apply(final List args) final Expr patternExpr = args.get(1); final Expr indexExpr = args.size() > 2 ? args.get(2) : null; - if (!patternExpr.isLiteral() || (indexExpr != null && !indexExpr.isLiteral())) { - throw new IAE("Function[%s] pattern and index must be literals", name()); + if (!ExprUtils.isStringLiteral(patternExpr)) { + throw new IAE("Function[%s] pattern must be a string literal", name()); + } + + if (indexExpr != null && (!indexExpr.isLiteral() || !(indexExpr.getLiteralValue() instanceof Number))) { + throw new IAE("Function[%s] index must be a numeric literal", name()); } // Precompile the pattern. - final Pattern pattern = Pattern.compile(String.valueOf(patternExpr.getLiteralValue())); + final Pattern pattern = Pattern.compile( + StringUtils.nullToEmptyNonDruidDataString((String) patternExpr.getLiteralValue()) + ); final int index = indexExpr == null ? 0 : ((Number) indexExpr.getLiteralValue()).intValue(); @@ -72,10 +78,16 @@ private RegexpExtractExpr(Expr arg) @Override public ExprEval eval(final ObjectBinding bindings) { - String s = arg.eval(bindings).asString(); - final Matcher matcher = pattern.matcher(NullHandling.nullToEmptyIfNeeded(s)); - final String retVal = matcher.find() ? matcher.group(index) : null; - return ExprEval.of(NullHandling.emptyToNullIfNeeded(retVal)); + final String s = NullHandling.nullToEmptyIfNeeded(arg.eval(bindings).asString()); + + if (s == null) { + // True nulls do not match anything. Note: this branch only executes in SQL-compatible null handling mode. + return ExprEval.of(null); + } else { + final Matcher matcher = pattern.matcher(NullHandling.nullToEmptyIfNeeded(s)); + final String retVal = matcher.find() ? matcher.group(index) : null; + return ExprEval.of(retVal); + } } @Override diff --git a/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java b/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java new file mode 100644 index 000000000000..83735e863494 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/expression/RegexpLikeExprMacro.java @@ -0,0 +1,101 @@ +/* + * 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.expression; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.math.expr.ExprType; + +import javax.annotation.Nonnull; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class RegexpLikeExprMacro implements ExprMacroTable.ExprMacro +{ + private static final String FN_NAME = "regexp_like"; + + @Override + public String name() + { + return FN_NAME; + } + + @Override + public Expr apply(final List args) + { + if (args.size() != 2) { + throw new IAE("Function[%s] must have 2 arguments", name()); + } + + final Expr arg = args.get(0); + final Expr patternExpr = args.get(1); + + if (!ExprUtils.isStringLiteral(patternExpr)) { + throw new IAE("Function[%s] pattern must be a string literal", name()); + } + + // Precompile the pattern. + final Pattern pattern = Pattern.compile( + StringUtils.nullToEmptyNonDruidDataString((String) patternExpr.getLiteralValue()) + ); + + class RegexpLikeExpr extends ExprMacroTable.BaseScalarUnivariateMacroFunctionExpr + { + private RegexpLikeExpr(Expr arg) + { + super(FN_NAME, arg); + } + + @Nonnull + @Override + public ExprEval eval(final ObjectBinding bindings) + { + final String s = NullHandling.nullToEmptyIfNeeded(arg.eval(bindings).asString()); + + if (s == null) { + // True nulls do not match anything. Note: this branch only executes in SQL-compatible null handling mode. + return ExprEval.of(false, ExprType.LONG); + } else { + final Matcher matcher = pattern.matcher(s); + return ExprEval.of(matcher.find(), ExprType.LONG); + } + } + + @Override + public Expr visit(Shuttle shuttle) + { + Expr newArg = arg.visit(shuttle); + return shuttle.visit(new RegexpLikeExpr(newArg)); + } + + @Override + public String stringify() + { + return StringUtils.format("%s(%s, %s)", FN_NAME, arg.stringify(), patternExpr.stringify()); + } + } + return new RegexpLikeExpr(arg); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/BitmapIndexSelector.java b/processing/src/main/java/org/apache/druid/query/filter/BitmapIndexSelector.java index 90307eb6380a..fd90e7412b68 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/BitmapIndexSelector.java +++ b/processing/src/main/java/org/apache/druid/query/filter/BitmapIndexSelector.java @@ -24,6 +24,7 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.spatial.ImmutableRTree; import org.apache.druid.segment.column.BitmapIndex; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.CloseableIndexed; import javax.annotation.Nullable; @@ -35,7 +36,7 @@ public interface BitmapIndexSelector @MustBeClosed @Nullable CloseableIndexed getDimensionValues(String dimension); - boolean hasMultipleValues(String dimension); + ColumnCapabilities.Capable hasMultipleValues(String dimension); int getNumRows(); BitmapFactory getBitmapFactory(); @Nullable diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java index 65af27b83fc2..efa0236acfdb 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java @@ -23,13 +23,11 @@ public class BooleanVectorValueMatcher extends BaseVectorValueMatcher { - private final VectorSizeInspector selector; private final boolean matches; private BooleanVectorValueMatcher(final VectorSizeInspector selector, final boolean matches) { super(selector); - this.selector = selector; this.matches = matches; } @@ -38,18 +36,6 @@ public static BooleanVectorValueMatcher of(final VectorSizeInspector selector, f return new BooleanVectorValueMatcher(selector, matches); } - @Override - public int getCurrentVectorSize() - { - return selector.getCurrentVectorSize(); - } - - @Override - public int getMaxVectorSize() - { - return selector.getCurrentVectorSize(); - } - @Override public ReadableVectorMatch match(final ReadableVectorMatch mask) { diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java index 7bd2a847c081..462644425dc1 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQuery.java @@ -249,8 +249,8 @@ private List> verifySubtotalsSpec( return subtotalsSpec; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java index 28399e0dbd35..f19ae25035ae 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2.java @@ -131,47 +131,53 @@ public static Sequence process( final ResourceHolder bufferHolder = intermediateResultsBufferPool.take(); - final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded( - query.getContextValue(GroupByStrategyV2.CTX_KEY_FUDGE_TIMESTAMP, null) - ); - - final DateTime fudgeTimestamp = fudgeTimestampString == null - ? null - : DateTimes.utc(Long.parseLong(fudgeTimestampString)); + try { + final String fudgeTimestampString = NullHandling.emptyToNullIfNeeded( + query.getContextValue(GroupByStrategyV2.CTX_KEY_FUDGE_TIMESTAMP, null) + ); - final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); - final Interval interval = Iterables.getOnlyElement(query.getIntervals()); + final DateTime fudgeTimestamp = fudgeTimestampString == null + ? null + : DateTimes.utc(Long.parseLong(fudgeTimestampString)); - final boolean doVectorize = queryConfig.getVectorize().shouldVectorize( - VectorGroupByEngine.canVectorize(query, storageAdapter, filter) - ); + final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getFilter())); + final Interval interval = Iterables.getOnlyElement(query.getIntervals()); - final Sequence result; - - if (doVectorize) { - result = VectorGroupByEngine.process( - query, - storageAdapter, - bufferHolder.get(), - fudgeTimestamp, - filter, - interval, - querySpecificConfig, - queryConfig + final boolean doVectorize = queryConfig.getVectorize().shouldVectorize( + VectorGroupByEngine.canVectorize(query, storageAdapter, filter) ); - } else { - result = processNonVectorized( - query, - storageAdapter, - bufferHolder.get(), - fudgeTimestamp, - querySpecificConfig, - filter, - interval - ); - } - return result.withBaggage(bufferHolder); + final Sequence result; + + if (doVectorize) { + result = VectorGroupByEngine.process( + query, + storageAdapter, + bufferHolder.get(), + fudgeTimestamp, + filter, + interval, + querySpecificConfig, + queryConfig + ); + } else { + result = processNonVectorized( + query, + storageAdapter, + bufferHolder.get(), + fudgeTimestamp, + querySpecificConfig, + filter, + interval + ); + } + + return result.withBaggage(bufferHolder); + } + catch (Throwable e) { + bufferHolder.close(); + throw e; + } } private static Sequence processNonVectorized( @@ -315,13 +321,13 @@ public static int getCardinalityForArrayAggregation( /** * Checks whether all "dimensions" are either single-valued, or if allowed, nonexistent. Since non-existent column * selectors will show up as full of nulls they are effectively single valued, however they can also be null during - * broker merge, for example with an 'inline' datasource subquery. 'missingMeansNonexistent' is sort of a hack to let + * broker merge, for example with an 'inline' datasource subquery. 'missingMeansNonExistent' is sort of a hack to let * the vectorized engine, which only operates on actual segments, to still work in this case for non-existent columns. */ public static boolean isAllSingleValueDims( final Function capabilitiesFunction, final List dimensions, - final boolean missingMeansNonexistent + final boolean missingMeansNonExistent ) { return dimensions @@ -336,8 +342,8 @@ public static boolean isAllSingleValueDims( // Now check column capabilities. final ColumnCapabilities columnCapabilities = capabilitiesFunction.apply(dimension.getDimension()); - return (columnCapabilities != null && !columnCapabilities.hasMultipleValues()) || - (missingMeansNonexistent && columnCapabilities == null); + return (columnCapabilities != null && !columnCapabilities.hasMultipleValues().isMaybeTrue()) || + (missingMeansNonExistent && columnCapabilities == null); }); } @@ -965,13 +971,13 @@ public Grouper.BufferComparator bufferComparatorWithAggregators( DefaultLimitSpec limitSpec = (DefaultLimitSpec) query.getLimitSpec(); return GrouperBufferComparatorUtils.bufferComparatorWithAggregators( - query.getAggregatorSpecs().toArray(new AggregatorFactory[0]), - aggregatorOffsets, - limitSpec, - query.getDimensions(), - getDimensionComparators(limitSpec), - query.getResultRowHasTimestamp(), - query.getContextSortByDimsFirst() + query.getAggregatorSpecs().toArray(new AggregatorFactory[0]), + aggregatorOffsets, + limitSpec, + query.getDimensions(), + getDimensionComparators(limitSpec), + query.getResultRowHasTimestamp(), + query.getContextSortByDimsFirst() ); } 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 3fa85040cea4..10408fe2d7a8 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 @@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.query.QueryConfig; import org.apache.druid.query.aggregation.AggregatorAdapters; -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.query.groupby.GroupByQuery; @@ -85,7 +84,7 @@ public static boolean canVectorize( return GroupByQueryEngineV2.isAllSingleValueDims(adapter::getColumnCapabilities, query.getDimensions(), true) && query.getDimensions().stream().allMatch(DimensionSpec::canVectorize) - && query.getAggregatorSpecs().stream().allMatch(AggregatorFactory::canVectorize) + && query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(adapter)) && adapter.canVectorize(filter, query.getVirtualColumns(), false); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparator.java b/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparator.java index 88f50efb8488..e3f786e1f8aa 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparator.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparator.java @@ -19,6 +19,7 @@ package org.apache.druid.query.groupby.having; +import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Doubles; import com.google.common.primitives.Longs; import org.apache.druid.java.util.common.ISE; @@ -89,10 +90,16 @@ static int compare(String aggregationName, Number value, Map toDrop ) { - this.current = current == null ? Collections.EMPTY_MAP : current; - this.toLoad = toLoad == null ? Collections.EMPTY_MAP : toLoad; - this.toDrop = toDrop == null ? Collections.EMPTY_SET : toDrop; + this.current = current == null ? Collections.emptyMap() : current; + this.toLoad = toLoad == null ? Collections.emptyMap() : toLoad; + this.toDrop = toDrop == null ? Collections.emptySet() : toDrop; } @JsonProperty 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 c5f1800acd15..659b55b1680d 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 @@ -45,6 +45,7 @@ import org.apache.druid.segment.column.DictionaryEncodedColumn; 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.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; @@ -101,9 +102,18 @@ public Map analyze(Segment segment) for (String columnName : columnNames) { final ColumnHolder columnHolder = index == null ? null : index.getColumnHolder(columnName); - final ColumnCapabilities capabilities = columnHolder != null - ? columnHolder.getCapabilities() - : storageAdapter.getColumnCapabilities(columnName); + final ColumnCapabilities capabilities; + if (columnHolder != null) { + capabilities = columnHolder.getCapabilities(); + } else { + // this can be removed if we get to the point where IncrementalIndexStorageAdapter.getColumnCapabilities + // accurately reports the capabilities + if (storageAdapter instanceof IncrementalIndexStorageAdapter) { + capabilities = ((IncrementalIndexStorageAdapter) storageAdapter).getSnapshotColumnCapabilities(columnName); + } else { + capabilities = storageAdapter.getColumnCapabilities(columnName); + } + } final ColumnAnalysis analysis; final ValueType type = capabilities.getType(); @@ -138,7 +148,7 @@ public Map analyze(Segment segment) // Add time column too ColumnCapabilities timeCapabilities = storageAdapter.getColumnCapabilities(ColumnHolder.TIME_COLUMN_NAME); if (timeCapabilities == null) { - timeCapabilities = new ColumnCapabilitiesImpl().setType(ValueType.LONG).setHasMultipleValues(false); + timeCapabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); } columns.put( ColumnHolder.TIME_COLUMN_NAME, @@ -172,7 +182,7 @@ private ColumnAnalysis analyzeNumericColumn( long size = 0; if (analyzingSize()) { - if (capabilities.hasMultipleValues()) { + if (capabilities.hasMultipleValues().isTrue()) { return ColumnAnalysis.error("multi_value"); } @@ -181,7 +191,7 @@ private ColumnAnalysis analyzeNumericColumn( return new ColumnAnalysis( capabilities.getType().name(), - capabilities.hasMultipleValues(), + capabilities.hasMultipleValues().isTrue(), size, null, null, @@ -231,7 +241,7 @@ private ColumnAnalysis analyzeStringColumn( return new ColumnAnalysis( capabilities.getType().name(), - capabilities.hasMultipleValues(), + capabilities.hasMultipleValues().isTrue(), size, analyzingCardinality() ? cardinality : 0, min, @@ -308,7 +318,7 @@ public Long accumulate(Long accumulated, Cursor cursor) return new ColumnAnalysis( capabilities.getType().name(), - capabilities.hasMultipleValues(), + capabilities.hasMultipleValues().isTrue(), size, cardinality, min, @@ -324,7 +334,7 @@ private ColumnAnalysis analyzeComplexColumn( ) { try (final ComplexColumn complexColumn = columnHolder != null ? (ComplexColumn) columnHolder.getColumn() : null) { - final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues(); + final boolean hasMultipleValues = capabilities != null && capabilities.hasMultipleValues().isTrue(); long size = 0; if (analyzingSize() && complexColumn != null) { 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 4237e50dc473..5b34f7623fed 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 @@ -205,7 +205,7 @@ public List getPreJoinableClauses() /** * Returns true if all servers have the ability to compute this datasource. These datasources depend only on - * globally broadcast data, like lookups or inline data. + * globally broadcast data, like lookups or inline data or broadcast segments. */ public boolean isGlobal() { diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java index 459b8326d217..be5d24b71b34 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQuery.java @@ -183,8 +183,8 @@ private Integer validateAndGetMaxSegmentPartitionsOrderedInMemory() return maxSegmentPartitionsOrderedInMemory; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java index 8233315d5e83..52066ab1f25e 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryRunnerFactory.java @@ -46,6 +46,7 @@ import org.apache.druid.segment.Segment; import org.joda.time.Interval; +import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Collections; @@ -83,7 +84,7 @@ public QueryRunner createRunner(Segment segment) @Override public QueryRunner mergeRunners( - ExecutorService queryExecutor, + final ExecutorService queryExecutor, final Iterable> queryRunners ) { @@ -122,14 +123,19 @@ public QueryRunner mergeRunners( : query.getMaxRowsQueuedForOrdering()); if (query.getScanRowsLimit() <= maxRowsQueuedForOrdering) { // Use priority queue strategy - return priorityQueueSortAndLimit( - Sequences.concat(Sequences.map( - Sequences.simple(queryRunnersOrdered), - input -> input.run(queryPlus, responseContext) - )), - query, - intervalsOrdered - ); + try { + return priorityQueueSortAndLimit( + Sequences.concat(Sequences.map( + Sequences.simple(queryRunnersOrdered), + input -> input.run(queryPlus, responseContext) + )), + query, + intervalsOrdered + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } } else { // Use n-way merge strategy List>> intervalsAndRunnersOrdered = new ArrayList<>(); @@ -149,11 +155,11 @@ public QueryRunner mergeRunners( // query runners for that segment LinkedHashMap>>> partitionsGroupedByInterval = intervalsAndRunnersOrdered.stream() - .collect(Collectors.groupingBy( - x -> x.lhs, - LinkedHashMap::new, - Collectors.toList() - )); + .collect(Collectors.groupingBy( + x -> x.lhs, + LinkedHashMap::new, + Collectors.toList() + )); // Find the segment with the largest numbers of partitions. This will be used to compare with the // maxSegmentPartitionsOrderedInMemory limit to determine if the query is at risk of consuming too much memory. @@ -203,7 +209,7 @@ Sequence priorityQueueSortAndLimit( Sequence inputSequence, ScanQuery scanQuery, List intervalsOrdered - ) + ) throws IOException { Comparator priorityQComparator = new ScanResultValueTimestampComparator(scanQuery); @@ -232,48 +238,54 @@ public ScanResultValue accumulate(ScanResultValue accumulated, ScanResultValue i } } ); - boolean doneScanning = yielder.isDone(); - // We need to scan limit elements and anything else in the last segment - int numRowsScanned = 0; - Interval finalInterval = null; - while (!doneScanning) { - ScanResultValue next = yielder.get(); - List singleEventScanResultValues = next.toSingleEventScanResultValues(); - for (ScanResultValue srv : singleEventScanResultValues) { - numRowsScanned++; - // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list - // needs to be preserved for queries using the compactedList result format - q.offer(srv); - if (q.size() > limit) { - q.poll(); - } - // Finish scanning the interval containing the limit row - if (numRowsScanned > limit && finalInterval == null) { - long timestampOfLimitRow = srv.getFirstEventTimestamp(scanQuery.getResultFormat()); - for (Interval interval : intervalsOrdered) { - if (interval.contains(timestampOfLimitRow)) { - finalInterval = interval; - } + try { + boolean doneScanning = yielder.isDone(); + // We need to scan limit elements and anything else in the last segment + int numRowsScanned = 0; + Interval finalInterval = null; + while (!doneScanning) { + ScanResultValue next = yielder.get(); + List singleEventScanResultValues = next.toSingleEventScanResultValues(); + for (ScanResultValue srv : singleEventScanResultValues) { + numRowsScanned++; + // Using an intermediate unbatched ScanResultValue is not that great memory-wise, but the column list + // needs to be preserved for queries using the compactedList result format + q.offer(srv); + if (q.size() > limit) { + q.poll(); } - if (finalInterval == null) { - throw new ISE("WTH??? Row came from an unscanned interval?"); + + // Finish scanning the interval containing the limit row + if (numRowsScanned > limit && finalInterval == null) { + long timestampOfLimitRow = srv.getFirstEventTimestamp(scanQuery.getResultFormat()); + for (Interval interval : intervalsOrdered) { + if (interval.contains(timestampOfLimitRow)) { + finalInterval = interval; + } + } + if (finalInterval == null) { + throw new ISE("WTH??? Row came from an unscanned interval?"); + } } } + yielder = yielder.next(null); + doneScanning = yielder.isDone() || + (finalInterval != null && + !finalInterval.contains(next.getFirstEventTimestamp(scanQuery.getResultFormat()))); + } + // Need to convert to a Deque because Priority Queue's iterator doesn't guarantee that the sorted order + // will be maintained. Deque was chosen over list because its addFirst is O(1). + final Deque sortedElements = new ArrayDeque<>(q.size()); + while (q.size() != 0) { + // addFirst is used since PriorityQueue#poll() dequeues the low-priority (timestamp-wise) events first. + sortedElements.addFirst(q.poll()); } - yielder = yielder.next(null); - doneScanning = yielder.isDone() || - (finalInterval != null && - !finalInterval.contains(next.getFirstEventTimestamp(scanQuery.getResultFormat()))); + return Sequences.simple(sortedElements); } - // Need to convert to a Deque because Priority Queue's iterator doesn't guarantee that the sorted order - // will be maintained. Deque was chosen over list because its addFirst is O(1). - final Deque sortedElements = new ArrayDeque<>(q.size()); - while (q.size() != 0) { - // addFirst is used since PriorityQueue#poll() dequeues the low-priority (timestamp-wise) events first. - sortedElements.addFirst(q.poll()); + finally { + yielder.close(); } - return Sequences.simple(sortedElements); } @VisibleForTesting 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 729d839d0298..6603ee05a519 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 @@ -50,6 +50,11 @@ public class TimeseriesQuery extends BaseQuery> { public static final String CTX_GRAND_TOTAL = "grandTotal"; public static final String SKIP_EMPTY_BUCKETS = "skipEmptyBuckets"; + // "timestampResultField" is an undocumented parameter used internally by the SQL layer. + // It is necessary because when the SQL layer generates a Timeseries query for a group-by-time-floor SQL query, + // it expects the result of the time-floor to have a specific name. That name is provided using this parameter. + // TODO: We can remove this once https://github.com/apache/druid/issues/9974 is done. + public static final String CTX_TIMESTAMP_RESULT_FIELD = "timestampResultField"; private final VirtualColumns virtualColumns; private final DimFilter dimFilter; @@ -103,8 +108,8 @@ public String getType() return Query.TIMESERIES; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; @@ -139,6 +144,11 @@ public boolean isGrandTotal() return getContextBoolean(CTX_GRAND_TOTAL, false); } + public String getTimestampResultField() + { + return getContextValue(CTX_TIMESTAMP_RESULT_FIELD); + } + public boolean isSkipEmptyBuckets() { return getContextBoolean(SKIP_EMPTY_BUCKETS, false); 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 1be36b2f10ef..f420f782f805 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 @@ -102,7 +102,7 @@ public Sequence> process(final TimeseriesQuery que final boolean doVectorize = queryConfigToUse.getVectorize().shouldVectorize( adapter.canVectorize(filter, query.getVirtualColumns(), descending) - && query.getAggregatorSpecs().stream().allMatch(AggregatorFactory::canVectorize) + && query.getAggregatorSpecs().stream().allMatch(aggregatorFactory -> aggregatorFactory.canVectorize(adapter)) ); final Sequence> result; diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index e381ffe86814..2ce7e3b34632 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -28,6 +28,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.inject.Inject; +import org.apache.commons.lang.StringUtils; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.granularity.Granularities; @@ -51,6 +52,7 @@ import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; import org.joda.time.DateTime; import java.util.Collections; @@ -286,6 +288,7 @@ public byte[] computeResultLevelCacheKey(TimeseriesQuery query) .appendCacheable(query.getVirtualColumns()) .appendCacheables(query.getPostAggregatorSpecs()) .appendInt(query.getLimit()) + .appendString(query.getTimestampResultField()) .appendBoolean(query.isGrandTotal()); return builder.build(); } @@ -404,11 +407,15 @@ public Function, Result> ma @Override public RowSignature resultArraySignature(TimeseriesQuery query) { - return RowSignature.builder() - .addTimeColumn() - .addAggregators(query.getAggregatorSpecs()) - .addPostAggregators(query.getPostAggregatorSpecs()) - .build(); + + RowSignature.Builder rowSignatureBuilder = RowSignature.builder(); + rowSignatureBuilder.addTimeColumn(); + if (StringUtils.isNotEmpty(query.getTimestampResultField())) { + rowSignatureBuilder.add(query.getTimestampResultField(), ValueType.LONG); + } + rowSignatureBuilder.addAggregators(query.getAggregatorSpecs()); + rowSignatureBuilder.addPostAggregators(query.getPostAggregatorSpecs()); + return rowSignatureBuilder.build(); } @Override @@ -447,13 +454,22 @@ private Function, Result> m return result -> { final TimeseriesResultValue holder = result.getValue(); final Map values = new HashMap<>(holder.getBaseObject()); - if (calculatePostAggs && !query.getPostAggregatorSpecs().isEmpty()) { - // put non finalized aggregators for calculating dependent post Aggregators - for (AggregatorFactory agg : query.getAggregatorSpecs()) { - values.put(agg.getName(), holder.getMetric(agg.getName())); + if (calculatePostAggs) { + if (!query.getPostAggregatorSpecs().isEmpty()) { + // put non finalized aggregators for calculating dependent post Aggregators + for (AggregatorFactory agg : query.getAggregatorSpecs()) { + values.put(agg.getName(), holder.getMetric(agg.getName())); + } + for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { + values.put(postAgg.getName(), postAgg.compute(values)); + } } - for (PostAggregator postAgg : query.getPostAggregatorSpecs()) { - values.put(postAgg.getName(), postAgg.compute(values)); + // If "timestampResultField" is set, we must include a copy of the timestamp in the result. + // This is used by the SQL layer when it generates a Timeseries query for a group-by-time-floor SQL query. + // The SQL layer expects the result of the time-floor to have a specific name that is not going to be "__time". + if (StringUtils.isNotEmpty(query.getTimestampResultField()) && result.getTimestamp() != null) { + final DateTime timestamp = result.getTimestamp(); + values.put(query.getTimestampResultField(), timestamp.getMillis()); } } for (AggregatorFactory agg : query.getAggregatorSpecs()) { 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 270ec1640866..ab5eef290851 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 @@ -36,6 +36,13 @@ import java.util.List; /** + * This {@link TopNAlgorithm} is tailored to processing aggregates on high cardility columns which are likely to have + * larger result sets. Internally it uses a 2 phase approach to compute the top-n result using the + * {@link PooledTopNAlgorithm} for each phase. The first phase is to process the segment with only the order-by + * aggregator to compute which values constitute the top 'n' results. With this information, a actual result set + * is computed by a second run of the {@link PooledTopNAlgorithm}, this time with all aggregators, but only considering + * the values from the 'n' results to avoid performing any aggregations that would have been thrown away for results + * that didn't make the top-n. */ public class AggregateTopNMetricFirstAlgorithm implements TopNAlgorithm { 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 c546b6f97c67..19ebb543578a 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 @@ -36,6 +36,7 @@ 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; import org.apache.druid.segment.historical.HistoricalColumnSelector; @@ -49,6 +50,18 @@ import java.util.List; /** + * This {@link TopNAlgorithm} is highly specialized for processing aggregates on string columns that are + * {@link ColumnCapabilities#isDictionaryEncoded()} and {@link ColumnCapabilities#areDictionaryValuesUnique()}. This + * algorithm is built around using a direct {@link ByteBuffer} from the 'processing pool' of intermediary results + * buffers, to aggregate using the dictionary id directly as the key, to defer looking up the value until is necessary. + * + * At runtime, this implementation is specialized with wizardry to optimize for processing common top-n query shapes, + * see {@link #computeSpecializedScanAndAggregateImplementations}, + * {@link Generic1AggPooledTopNScanner} and {@link Generic1AggPooledTopNScannerPrototype}, + * {@link Generic2AggPooledTopNScanner} and {@link Generic2AggPooledTopNScannerPrototype}, + * {@link org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop}, + * {@link org.apache.druid.query.monomorphicprocessing.HotLoopCallee}, + * {@link org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector} for more details. */ public class PooledTopNAlgorithm extends BaseTopNAlgorithm @@ -211,10 +224,6 @@ public PooledTopNAlgorithm( @Override public PooledTopNParams makeInitParams(ColumnSelectorPlus selectorPlus, Cursor cursor) { - ResourceHolder resultsBufHolder = bufferPool.take(); - ByteBuffer resultsBuf = resultsBufHolder.get(); - resultsBuf.clear(); - final DimensionSelector dimSelector = (DimensionSelector) selectorPlus.getSelector(); final int cardinality = dimSelector.getValueCardinality(); @@ -243,27 +252,38 @@ public int[] build() } }; - final int numBytesToWorkWith = resultsBuf.remaining(); - final int[] aggregatorSizes = new int[query.getAggregatorSpecs().size()]; - int numBytesPerRecord = 0; + final ResourceHolder resultsBufHolder = bufferPool.take(); - for (int i = 0; i < query.getAggregatorSpecs().size(); ++i) { - aggregatorSizes[i] = query.getAggregatorSpecs().get(i).getMaxIntermediateSizeWithNulls(); - numBytesPerRecord += aggregatorSizes[i]; - } + try { + final ByteBuffer resultsBuf = resultsBufHolder.get(); + resultsBuf.clear(); + + final int numBytesToWorkWith = resultsBuf.remaining(); + final int[] aggregatorSizes = new int[query.getAggregatorSpecs().size()]; + int numBytesPerRecord = 0; - final int numValuesPerPass = numBytesPerRecord > 0 ? numBytesToWorkWith / numBytesPerRecord : cardinality; - - return PooledTopNParams.builder() - .withSelectorPlus(selectorPlus) - .withCursor(cursor) - .withResultsBufHolder(resultsBufHolder) - .withResultsBuf(resultsBuf) - .withArrayProvider(arrayProvider) - .withNumBytesPerRecord(numBytesPerRecord) - .withNumValuesPerPass(numValuesPerPass) - .withAggregatorSizes(aggregatorSizes) - .build(); + for (int i = 0; i < query.getAggregatorSpecs().size(); ++i) { + aggregatorSizes[i] = query.getAggregatorSpecs().get(i).getMaxIntermediateSizeWithNulls(); + numBytesPerRecord += aggregatorSizes[i]; + } + + final int numValuesPerPass = numBytesPerRecord > 0 ? numBytesToWorkWith / numBytesPerRecord : cardinality; + + return PooledTopNParams.builder() + .withSelectorPlus(selectorPlus) + .withCursor(cursor) + .withResultsBufHolder(resultsBufHolder) + .withResultsBuf(resultsBuf) + .withArrayProvider(arrayProvider) + .withNumBytesPerRecord(numBytesPerRecord) + .withNumValuesPerPass(numValuesPerPass) + .withAggregatorSizes(aggregatorSizes) + .build(); + } + catch (Throwable e) { + resultsBufHolder.close(); + throw e; + } } diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java index 3c301e584af8..f5bdec1982a0 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQuery.java @@ -113,8 +113,8 @@ public String getType() return TOPN; } - @Override @JsonProperty + @Override public VirtualColumns getVirtualColumns() { return virtualColumns; 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 1d1bccdd7b1b..eb22dc9b4ae6 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 @@ -53,6 +53,12 @@ public TopNQueryEngine(NonBlockingPool bufferPool) this.bufferPool = bufferPool; } + /** + * Do the thing - process a {@link StorageAdapter} 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 org.apache.druid.segment.Cursor} to apply + * {@link AggregatorFactory} and create or update {@link TopNResultValue} + */ public Sequence> query( final TopNQuery query, final StorageAdapter adapter, @@ -71,7 +77,9 @@ public Sequence> query( final TopNMapFn mapFn = getMapFn(query, adapter, queryMetrics); Preconditions.checkArgument( - queryIntervals.size() == 1, "Can only handle a single interval, got[%s]", queryIntervals + queryIntervals.size() == 1, + "Can only handle a single interval, got[%s]", + queryIntervals ); return Sequences.filter( @@ -95,6 +103,9 @@ public Sequence> query( ); } + /** + * Choose the best {@link TopNAlgorithm} for the given query. + */ private TopNMapFn getMapFn( final TopNQuery query, final StorageAdapter adapter, @@ -120,36 +131,34 @@ private TopNMapFn getMapFn( final TopNAlgorithm topNAlgorithm; - if ( - selector.isHasExtractionFn() && - // TimeExtractionTopNAlgorithm can work on any single-value dimension of type long. - // Once we have arbitrary dimension types following check should be replaced by checking - // that the column is of type long and single-value. - dimension.equals(ColumnHolder.TIME_COLUMN_NAME) - ) { - // A special TimeExtractionTopNAlgorithm is required, since DimExtractionTopNAlgorithm - // currently relies on the dimension cardinality to support lexicographic sorting - topNAlgorithm = new TimeExtractionTopNAlgorithm(adapter, query); - } else if (selector.isHasExtractionFn()) { - topNAlgorithm = new HeapBasedTopNAlgorithm(adapter, query); - } else if (columnCapabilities == null || !(columnCapabilities.getType() == ValueType.STRING - && columnCapabilities.isDictionaryEncoded())) { - // Use HeapBasedTopNAlgorithm for non-Strings and for non-dictionary-encoded Strings, and for things we don't know - // which can happen for 'inline' data sources when this is run on the broker - topNAlgorithm = new HeapBasedTopNAlgorithm(adapter, query); - } else if (query.getDimensionSpec().getOutputType() != ValueType.STRING) { - // Use HeapBasedTopNAlgorithm when the dimension output type is a non-String. (It's like an extractionFn: there can be - // a many-to-one mapping, since numeric types can't represent all possible values of other types.) - topNAlgorithm = new HeapBasedTopNAlgorithm(adapter, query); - } else if (selector.isAggregateAllMetrics()) { - // sorted by dimension - topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool); - } else if (selector.isAggregateTopNMetricFirst() || query.getContextBoolean("doAggregateTopNMetricFirst", false)) { - // high cardinality dimensions with larger result sets - topNAlgorithm = new AggregateTopNMetricFirstAlgorithm(adapter, query, bufferPool); + if (canUsePooledAlgorithm(selector, query, columnCapabilities)) { + // 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); + } else if (selector.isAggregateTopNMetricFirst() || query.getContextBoolean("doAggregateTopNMetricFirst", false)) { + // 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); + } else { + // anything else, use the regular pooled algorithm + topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool); + } } else { - // anything else - topNAlgorithm = new PooledTopNAlgorithm(adapter, query, bufferPool); + // heap based algorithm selection, if we must + if (selector.isHasExtractionFn() && dimension.equals(ColumnHolder.TIME_COLUMN_NAME)) { + // TimeExtractionTopNAlgorithm can work on any single-value dimension of type long. + // We might be able to use this for any long column with an extraction function, that is + // ValueType.LONG.equals(columnCapabilities.getType()) + // but this needs investigation to ensure that it is an improvement over HeapBasedTopNAlgorithm + + // A special TimeExtractionTopNAlgorithm is required since HeapBasedTopNAlgorithm + // currently relies on the dimension cardinality to support lexicographic sorting + topNAlgorithm = new TimeExtractionTopNAlgorithm(adapter, query); + } else { + topNAlgorithm = new HeapBasedTopNAlgorithm(adapter, query); + } } if (queryMetrics != null) { queryMetrics.algorithm(topNAlgorithm); @@ -158,6 +167,48 @@ private TopNMapFn getMapFn( return new TopNMapFn(query, topNAlgorithm); } + /** + * {@link PooledTopNAlgorithm} (and {@link AggregateTopNMetricFirstAlgorithm} which utilizes the pooled + * algorithm) are optimized off-heap algorithms for aggregating dictionary encoded string columns. These algorithms + * rely on dictionary ids being unique so to aggregate on the dictionary ids directly and defer + * {@link org.apache.druid.segment.DimensionSelector#lookupName(int)} until as late as possible in query processing. + * + * When these conditions are not true, we have an on-heap fall-back algorithm, the {@link HeapBasedTopNAlgorithm} + * (and {@link TimeExtractionTopNAlgorithm} for a specialized form for long columns) which aggregates on values of + * selectors. + */ + private static boolean canUsePooledAlgorithm( + final TopNAlgorithmSelector selector, + final TopNQuery query, + final ColumnCapabilities capabilities + ) + { + if (selector.isHasExtractionFn()) { + // extraction functions can have a many to one mapping, and should use a heap algorithm + return false; + } + + if (query.getDimensionSpec().getOutputType() != ValueType.STRING) { + // non-string output cannot use the pooled algorith, even if the underlying selector supports it + return false; + } + if (capabilities != null && capabilities.getType() == ValueType.STRING) { + // string columns must use the on heap algorithm unless they have the following capabilites + return capabilities.isDictionaryEncoded() && capabilities.areDictionaryValuesUnique().isTrue(); + } else { + // non-strings are not eligible to use the pooled algorithm, and should use a heap algorithm + return false; + } + } + + /** + * {@link ExtractionFn} which are one to one may have their execution deferred until as late as possible, since the + * which value is used as the grouping key itself doesn't particularly matter. For top-n, this method allows the + * query to be transformed in {@link TopNQueryQueryToolChest#preMergeQueryDecoration} to strip off the + * {@link ExtractionFn} on the broker, so that a more optimized algorithm (e.g. {@link PooledTopNAlgorithm}) can be + * chosen for processing segments, and then added back and evaluated against the final merged result sets on the + * broker via {@link TopNQueryQueryToolChest#postMergeQueryDecoration}. + */ public static boolean canApplyExtractionInPost(TopNQuery query) { return query.getDimensionSpec() != null diff --git a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java index cea7c775a758..92b2e8a3fe9c 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java +++ b/processing/src/main/java/org/apache/druid/query/topn/types/StringTopNColumnAggregatesProcessor.java @@ -59,7 +59,7 @@ public Aggregator[][] getRowSelector(TopNQuery query, TopNParams params, Storage throw new UnsupportedOperationException("Cannot operate on a dimension with unknown cardinality"); } - // This method is used for the DimExtractionTopNAlgorithm only. + // This method is used for the HeapBasedTopNAlgorithm only. // Unlike regular topN we cannot rely on ordering to optimize. // Optimization possibly requires a reverse lookup from value to ID, which is // not possible when applying an extraction function diff --git a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java b/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java index 1518c44dcf07..c41be41ec4d9 100644 --- a/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/AbstractSegment.java @@ -19,19 +19,11 @@ package org.apache.druid.segment; -import javax.annotation.Nullable; - +/** + * @deprecated use {@link Segment} directly as this does nothing + */ +@Deprecated public abstract class AbstractSegment implements Segment { - @Override - @Nullable - public T as(Class clazz) - { - if (clazz.equals(QueryableIndex.class)) { - return (T) asQueryableIndex(); - } else if (clazz.equals(StorageAdapter.class)) { - return (T) asStorageAdapter(); - } - return null; - } + // i used to have a purpose } diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCounter.java b/processing/src/main/java/org/apache/druid/segment/ColumnInspector.java similarity index 60% rename from processing/src/main/java/org/apache/druid/segment/ReferenceCounter.java rename to processing/src/main/java/org/apache/druid/segment/ColumnInspector.java index 970d487b7703..3090455833e3 100644 --- a/processing/src/main/java/org/apache/druid/segment/ReferenceCounter.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnInspector.java @@ -19,26 +19,19 @@ package org.apache.druid.segment; -import java.io.Closeable; +import org.apache.druid.segment.column.ColumnCapabilities; -/** - * An interface to reference-counted objects. Used by {@link ReferenceCountingSegment}. Thread-safe. - */ -public interface ReferenceCounter -{ - /** - * Increment the reference count by one. - */ - boolean increment(); - - /** - * Returns a {@link Closeable} which action is to call {@link #decrement()} only once. If close() is called on the - * returned Closeable object for the second time, it won't call {@link #decrement()} again. - */ - Closeable decrementOnceCloseable(); +import javax.annotation.Nullable; +public interface ColumnInspector +{ /** - * Decrement the reference count by one. + * Returns capabilities of a particular column. + * + * @param column column name + * + * @return capabilities, or null */ - void decrement(); + @Nullable + ColumnCapabilities getColumnCapabilities(String column); } diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java b/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java index 5fb698a484e4..05e85fdd4c24 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java @@ -197,6 +197,6 @@ private static T makeProcessorInternal( */ private static boolean mayBeMultiValue(@Nullable final ColumnCapabilities capabilities) { - return capabilities == null || !capabilities.isComplete() || capabilities.hasMultipleValues(); + return capabilities == null || capabilities.hasMultipleValues().isMaybeTrue(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorBitmapIndexSelector.java b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorBitmapIndexSelector.java index 79b6e8900745..bd6de7a02d57 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorBitmapIndexSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorBitmapIndexSelector.java @@ -27,6 +27,7 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.BaseColumn; import org.apache.druid.segment.column.BitmapIndex; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.column.NumericColumn; @@ -157,14 +158,18 @@ public void close() throws IOException } @Override - public boolean hasMultipleValues(final String dimension) + public ColumnCapabilities.Capable hasMultipleValues(final String dimension) { if (isVirtualColumn(dimension)) { return virtualColumns.getVirtualColumn(dimension).capabilities(dimension).hasMultipleValues(); } final ColumnHolder columnHolder = index.getColumnHolder(dimension); - return columnHolder != null && columnHolder.getCapabilities().hasMultipleValues(); + // if ColumnHolder is null, the column doesn't exist, but report as not having multiple values so that + // the empty bitmap will be used + return columnHolder != null + ? columnHolder.getCapabilities().hasMultipleValues() + : ColumnCapabilities.Capable.FALSE; } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java index 07e66a672259..f99f0eade621 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorFactory.java @@ -31,7 +31,7 @@ * @see org.apache.druid.segment.vector.VectorColumnSelectorFactory, the vectorized version */ @PublicApi -public interface ColumnSelectorFactory +public interface ColumnSelectorFactory extends ColumnInspector { DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec); @@ -50,6 +50,7 @@ public interface ColumnSelectorFactory * * @return capabilities, or null */ + @Override @Nullable ColumnCapabilities getColumnCapabilities(String column); } diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java index 0c8b3c280c6e..9e5a12921c7c 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionHandlerUtils.java @@ -289,15 +289,23 @@ public static T makeVectorProcessor( final VectorColumnSelectorFactory selectorFactory ) { - final ColumnCapabilities capabilities = getEffectiveCapabilities( + final ColumnCapabilities originalCapabilities = + selectorFactory.getColumnCapabilities(dimensionSpec.getDimension()); + + final ColumnCapabilities effectiveCapabilites = getEffectiveCapabilities( dimensionSpec, - selectorFactory.getColumnCapabilities(dimensionSpec.getDimension()) + originalCapabilities ); - final ValueType type = capabilities.getType(); + final ValueType type = effectiveCapabilites.getType(); + + // vector selectors should never have null column capabilities, these signify a non-existent column, and complex + // columns should never be treated as a multi-value column, so always use single value string processor + final boolean forceSingleValue = + originalCapabilities == null || ValueType.COMPLEX.equals(originalCapabilities.getType()); if (type == ValueType.STRING) { - if (capabilities.hasMultipleValues()) { + if (!forceSingleValue && effectiveCapabilites.hasMultipleValues().isMaybeTrue()) { return strategyFactory.makeMultiValueDimensionProcessor( selectorFactory.makeMultiValueDimensionSelector(dimensionSpec) ); @@ -328,7 +336,7 @@ public static T makeVectorProcessor( selectorFactory.makeValueSelector(dimensionSpec.getDimension()) ); } else { - throw new ISE("Unsupported type[%s]", capabilities.getType()); + throw new ISE("Unsupported type[%s]", effectiveCapabilites.getType()); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java index 99921eb79c08..cf7631db08bb 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionIndexer.java @@ -127,6 +127,19 @@ public interface DimensionIndexer */ EncodedKeyComponentType processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValues, boolean reportParseExceptions); + /** + * This method will be called while building an {@link IncrementalIndex} whenever a known dimension column (either + * through an explicit schema on the ingestion spec, or auto-discovered while processing rows) is absent in any row + * that is processed, to allow an indexer to account for any missing rows if necessary. Useful so that a string + * {@link DimensionSelector} built on top of an {@link IncrementalIndex} may accurately report + * {@link DimensionSelector#nameLookupPossibleInAdvance()} by allowing it to track if it has any implicit null valued + * rows. + * + * At index persist/merge time all missing columns for a row will be explicitly replaced with the value appropriate + * null or default value. + */ + void setSparseIndexed(); + /** * Gives the estimated size in bytes for the given key * diff --git a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java index 6645e1fc7328..b802f7555135 100644 --- a/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DoubleDimensionIndexer.java @@ -47,6 +47,12 @@ public Double processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimVa return DimensionHandlerUtils.convertObjectToDouble(dimValues, reportParseExceptions); } + @Override + public void setSparseIndexed() + { + // no-op, double columns do not have a dictionary to track null values + } + @Override public long estimateEncodedKeyComponentSize(Double key) { diff --git a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java index 41328828de2b..dce58a23b23f 100644 --- a/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/FloatDimensionIndexer.java @@ -48,6 +48,12 @@ public Float processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimVal return DimensionHandlerUtils.convertObjectToFloat(dimValues, reportParseExceptions); } + @Override + public void setSparseIndexed() + { + // no-op, float columns do not have a dictionary to track null values + } + @Override public long estimateEncodedKeyComponentSize(Float key) { 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 683106cf3871..b270b54c4d1d 100644 --- a/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/IncrementalIndexSegment.java @@ -26,7 +26,7 @@ /** */ -public class IncrementalIndexSegment extends AbstractSegment +public class IncrementalIndexSegment implements Segment { private final IncrementalIndex index; private final SegmentId segmentId; diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java index e1a92f7d1a36..066b8dc41ea6 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -49,6 +49,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; +import org.apache.druid.segment.loading.SegmentizerFactory; import org.apache.druid.segment.serde.ColumnPartSerde; import org.apache.druid.segment.serde.ComplexColumnPartSerde; import org.apache.druid.segment.serde.ComplexMetricSerde; @@ -152,14 +153,19 @@ private File makeIndexFiles( progress.progress(); startTime = System.currentTimeMillis(); try (FileOutputStream fos = new FileOutputStream(new File(outDir, "factory.json"))) { - mapper.writeValue(fos, new MMappedQueryableSegmentizerFactory(indexIO)); + SegmentizerFactory customSegmentLoader = indexSpec.getSegmentLoader(); + if (customSegmentLoader != null) { + mapper.writeValue(fos, customSegmentLoader); + } else { + mapper.writeValue(fos, new MMappedQueryableSegmentizerFactory(indexIO)); + } } log.debug("Completed factory.json in %,d millis", System.currentTimeMillis() - startTime); progress.progress(); final Map metricsValueTypes = new TreeMap<>(Comparators.naturalNullsFirst()); final Map metricTypeNames = new TreeMap<>(Comparators.naturalNullsFirst()); - final List dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size()); + final List dimCapabilities = Lists.newArrayListWithCapacity(mergedDimensions.size()); mergeCapabilities(adapters, mergedDimensions, metricsValueTypes, metricTypeNames, dimCapabilities); final Map handlers = makeDimensionHandlers(mergedDimensions, dimCapabilities); @@ -710,18 +716,22 @@ private void mergeCapabilities( final List mergedDimensions, final Map metricsValueTypes, final Map metricTypeNames, - final List dimCapabilities + final List dimCapabilities ) { - final Map capabilitiesMap = new HashMap<>(); + final Map capabilitiesMap = new HashMap<>(); for (IndexableAdapter adapter : adapters) { for (String dimension : adapter.getDimensionNames()) { ColumnCapabilities capabilities = adapter.getCapabilities(dimension); - capabilitiesMap.computeIfAbsent(dimension, d -> new ColumnCapabilitiesImpl().setIsComplete(true)).merge(capabilities); + capabilitiesMap.compute(dimension, (d, existingCapabilities) -> + ColumnCapabilitiesImpl.snapshot(capabilities) + .merge(ColumnCapabilitiesImpl.snapshot(existingCapabilities))); } for (String metric : adapter.getMetricNames()) { ColumnCapabilities capabilities = adapter.getCapabilities(metric); - capabilitiesMap.computeIfAbsent(metric, m -> new ColumnCapabilitiesImpl().setIsComplete(true)).merge(capabilities); + capabilitiesMap.compute(metric, (m, existingCapabilities) -> + ColumnCapabilitiesImpl.snapshot(capabilities) + .merge(ColumnCapabilitiesImpl.snapshot(existingCapabilities))); metricsValueTypes.put(metric, capabilities.getType()); metricTypeNames.put(metric, adapter.getMetricType(metric)); } @@ -996,7 +1006,7 @@ public File append( private Map makeDimensionHandlers( final List mergedDimensions, - final List dimCapabilities + final List dimCapabilities ) { Map handlers = new LinkedHashMap<>(); diff --git a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java index 6edc33686f23..13101de94db0 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexSpec.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexSpec.java @@ -21,12 +21,14 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.Sets; import org.apache.druid.segment.data.BitmapSerde; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; +import org.apache.druid.segment.loading.SegmentizerFactory; import javax.annotation.Nullable; import java.util.Arrays; @@ -62,13 +64,26 @@ public class IndexSpec private final CompressionStrategy metricCompression; private final CompressionFactory.LongEncodingStrategy longEncoding; + @Nullable + private final SegmentizerFactory segmentLoader; /** * Creates an IndexSpec with default parameters */ public IndexSpec() { - this(null, null, null, null); + this(null, null, null, null, null); + } + + @VisibleForTesting + public IndexSpec( + @Nullable BitmapSerdeFactory bitmapSerdeFactory, + @Nullable CompressionStrategy dimensionCompression, + @Nullable CompressionStrategy metricCompression, + @Nullable CompressionFactory.LongEncodingStrategy longEncoding + ) + { + this(bitmapSerdeFactory, dimensionCompression, metricCompression, longEncoding, null); } /** @@ -93,7 +108,8 @@ public IndexSpec( @JsonProperty("bitmap") @Nullable BitmapSerdeFactory bitmapSerdeFactory, @JsonProperty("dimensionCompression") @Nullable CompressionStrategy dimensionCompression, @JsonProperty("metricCompression") @Nullable CompressionStrategy metricCompression, - @JsonProperty("longEncoding") @Nullable CompressionFactory.LongEncodingStrategy longEncoding + @JsonProperty("longEncoding") @Nullable CompressionFactory.LongEncodingStrategy longEncoding, + @JsonProperty("segmentLoader") @Nullable SegmentizerFactory segmentLoader ) { Preconditions.checkArgument(dimensionCompression == null || DIMENSION_COMPRESSION.contains(dimensionCompression), @@ -111,6 +127,7 @@ public IndexSpec( this.dimensionCompression = dimensionCompression == null ? DEFAULT_DIMENSION_COMPRESSION : dimensionCompression; this.metricCompression = metricCompression == null ? DEFAULT_METRIC_COMPRESSION : metricCompression; this.longEncoding = longEncoding == null ? DEFAULT_LONG_ENCODING : longEncoding; + this.segmentLoader = segmentLoader; } @JsonProperty("bitmap") @@ -137,6 +154,13 @@ public CompressionFactory.LongEncodingStrategy getLongEncoding() return longEncoding; } + @JsonProperty + @Nullable + public SegmentizerFactory getSegmentLoader() + { + return segmentLoader; + } + @Override public boolean equals(Object o) { @@ -150,13 +174,14 @@ public boolean equals(Object o) return Objects.equals(bitmapSerdeFactory, indexSpec.bitmapSerdeFactory) && dimensionCompression == indexSpec.dimensionCompression && metricCompression == indexSpec.metricCompression && - longEncoding == indexSpec.longEncoding; + longEncoding == indexSpec.longEncoding && + Objects.equals(segmentLoader, indexSpec.segmentLoader); } @Override public int hashCode() { - return Objects.hash(bitmapSerdeFactory, dimensionCompression, metricCompression, longEncoding); + return Objects.hash(bitmapSerdeFactory, dimensionCompression, metricCompression, longEncoding, segmentLoader); } @Override @@ -167,6 +192,7 @@ public String toString() ", dimensionCompression=" + dimensionCompression + ", metricCompression=" + metricCompression + ", longEncoding=" + longEncoding + + ", segmentLoader=" + segmentLoader + '}'; } } diff --git a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java index d35724398f9a..f2a91278f6bb 100644 --- a/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/LongDimensionIndexer.java @@ -48,6 +48,12 @@ public Long processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimValu return DimensionHandlerUtils.convertObjectToLong(dimValues, reportParseExceptions); } + @Override + public void setSparseIndexed() + { + // no-op, long columns do not have a dictionary to track null values + } + @Override public long estimateEncodedKeyComponentSize(Long key) { 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 30b3fcba4df9..a829dfae184b 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java @@ -24,7 +24,7 @@ /** */ -public class QueryableIndexSegment extends AbstractSegment +public class QueryableIndexSegment implements Segment { private final QueryableIndex index; private final QueryableIndexStorageAdapter storageAdapter; diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountedObject.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountedObject.java new file mode 100644 index 000000000000..4770743f3bbe --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountedObject.java @@ -0,0 +1,46 @@ +/* + * 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 java.io.Closeable; +import java.util.Optional; + +/** + * Interface for an object that may have a reference acquired in the form of a {@link Closeable}. This is intended to be + * used with an implementation of {@link ReferenceCountingCloseableObject}, or anything else that wishes to provide + * a method to account for the acquire and release of a reference to the object. + */ +public interface ReferenceCountedObject +{ + /** + * This method is expected to increment a reference count and provide a {@link Closeable} that decrements the + * reference count when closed. This is likely just a wrapper around + * {@link ReferenceCountingCloseableObject#incrementReferenceAndDecrementOnceCloseable()}, but may also include any + * other associated references which should be incremented when this method is called, and decremented/released by the + * closeable. + * + * IMPORTANT NOTE: to fulfill the contract of this method, implementors must return a closeable to indicate that the + * reference can be acquired, even if there is nothing to close. Implementors should avoid allowing this method or the + * {@link Closeable} it creates to throw exceptions. + * + * For callers: if this method returns non-empty, IT MUST BE CLOSED, else reference counts can potentially leak. + */ + Optional acquireReferences(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingCloseableObject.java b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingCloseableObject.java new file mode 100644 index 000000000000..af0e1c7a3770 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingCloseableObject.java @@ -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. + */ + +package org.apache.druid.segment; + +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; + +import java.io.Closeable; +import java.util.Optional; +import java.util.concurrent.Phaser; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * ReferenceCountingCloseableObject implements something like automatic reference count-based resource management, + * backed by a {@link Phaser}. + * + * ReferenceCountingCloseableObject allows consumers to call {@link #close()} before some other "users", which called + * {@link #increment()} or {@link #incrementReferenceAndDecrementOnceCloseable()}, but have not called + * {@link #decrement()} yet or the closer for {@link #incrementReferenceAndDecrementOnceCloseable()}, and the wrapped + * object won't be actually closed until that all references are released. + */ +public abstract class ReferenceCountingCloseableObject implements Closeable +{ + private static final Logger log = new Logger(ReferenceCountingCloseableObject.class); + + private final AtomicBoolean closed = new AtomicBoolean(false); + protected final Phaser referents = new Phaser(1) + { + @Override + protected boolean onAdvance(int phase, int registeredParties) + { + // Ensure that onAdvance() doesn't throw exception, otherwise termination won't happen + if (registeredParties != 0) { + log.error("registeredParties[%s] is not 0", registeredParties); + } + try { + baseObject.close(); + } + catch (Exception e) { + try { + log.error(e, "Exception while closing reference counted object[%s]", baseObject); + } + catch (Exception e2) { + // ignore + } + } + // Always terminate. + return true; + } + }; + + protected final BaseObject baseObject; + + public ReferenceCountingCloseableObject(BaseObject object) + { + this.baseObject = object; + } + + public int getNumReferences() + { + return Math.max(referents.getRegisteredParties() - 1, 0); + } + + public boolean isClosed() + { + return referents.isTerminated(); + } + + /** + * Increment the reference count by one. + */ + public boolean increment() + { + // Negative return from referents.register() means the Phaser is terminated. + return referents.register() >= 0; + } + + /** + * Decrement the reference count by one. + */ + public void decrement() + { + referents.arriveAndDeregister(); + } + + /** + * Returns an {@link Optional} of a {@link Closeable} from {@link #decrementOnceCloseable}, if it is able to + * successfully {@link #increment}, else nothing indicating that the reference could not be acquired. + */ + public Optional incrementReferenceAndDecrementOnceCloseable() + { + final Closer closer; + if (increment()) { + closer = Closer.create(); + closer.register(decrementOnceCloseable()); + } else { + closer = null; + } + return Optional.ofNullable(closer); + } + + /** + * Returns a {@link Closeable} which action is to call {@link #decrement()} only once. If close() is called on the + * returned Closeable object for the second time, it won't call {@link #decrement()} again. + */ + public Closeable decrementOnceCloseable() + { + AtomicBoolean decremented = new AtomicBoolean(false); + return () -> { + if (decremented.compareAndSet(false, true)) { + decrement(); + } else { + log.warn("close() is called more than once on ReferenceCountingCloseableObject.decrementOnceCloseable()"); + } + }; + } + + @Override + public void close() + { + if (closed.compareAndSet(false, true)) { + referents.arriveAndDeregister(); + } else { + log.warn("close() is called more than once on ReferenceCountingCloseableObject"); + } + } +} 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 ba1c34318e33..5b531316cd5a 100644 --- a/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/ReferenceCountingSegment.java @@ -20,7 +20,6 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; -import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.timeline.Overshadowable; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.ShardSpec; @@ -28,50 +27,20 @@ import javax.annotation.Nullable; import java.io.Closeable; -import java.util.concurrent.Phaser; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.Optional; /** - * ReferenceCountingSegment allows to call {@link #close()} before some other "users", which called {@link - * #increment()}, has not called {@link #decrement()} yet, and the wrapped {@link Segment} won't be actually closed - * until that. So ReferenceCountingSegment implements something like automatic reference count-based resource - * management. + * {@link Segment} that is also a {@link ReferenceCountingSegment}, allowing query engines that operate directly on + * segments to track references so that dropping a {@link Segment} can be done safely to ensure there are no in-flight + * queries. */ -public class ReferenceCountingSegment extends AbstractSegment - implements Overshadowable, ReferenceCounter +public class ReferenceCountingSegment extends ReferenceCountingCloseableObject + implements SegmentReference, Overshadowable { - private static final EmittingLogger log = new EmittingLogger(ReferenceCountingSegment.class); - - private final Segment baseSegment; private final short startRootPartitionId; private final short endRootPartitionId; private final short minorVersion; private final short atomicUpdateGroupSize; - private final AtomicBoolean closed = new AtomicBoolean(false); - private final Phaser referents = new Phaser(1) - { - @Override - protected boolean onAdvance(int phase, int registeredParties) - { - // Ensure that onAdvance() doesn't throw exception, otherwise termination won't happen - if (registeredParties != 0) { - log.error("registeredParties[%s] is not 0", registeredParties); - } - try { - baseSegment.close(); - } - catch (Exception e) { - try { - log.error(e, "Exception while closing segment[%s]", baseSegment.getId()); - } - catch (Exception e2) { - // ignore - } - } - // Always terminate. - return true; - } - }; public static ReferenceCountingSegment wrapRootGenerationSegment(Segment baseSegment) { @@ -106,7 +75,7 @@ private ReferenceCountingSegment( short atomicUpdateGroupSize ) { - this.baseSegment = baseSegment; + super(baseSegment); this.startRootPartitionId = (short) startRootPartitionId; this.endRootPartitionId = (short) endRootPartitionId; this.minorVersion = minorVersion; @@ -116,105 +85,43 @@ private ReferenceCountingSegment( @Nullable public Segment getBaseSegment() { - return !isClosed() ? baseSegment : null; - } - - public int getNumReferences() - { - return Math.max(referents.getRegisteredParties() - 1, 0); - } - - public boolean isClosed() - { - return referents.isTerminated(); + return !isClosed() ? baseObject : null; } @Override @Nullable public SegmentId getId() { - return !isClosed() ? baseSegment.getId() : null; + return !isClosed() ? baseObject.getId() : null; } @Override @Nullable public Interval getDataInterval() { - return !isClosed() ? baseSegment.getDataInterval() : null; + return !isClosed() ? baseObject.getDataInterval() : null; } @Override @Nullable public QueryableIndex asQueryableIndex() { - return !isClosed() ? baseSegment.asQueryableIndex() : null; + return !isClosed() ? baseObject.asQueryableIndex() : null; } @Override @Nullable public StorageAdapter asStorageAdapter() { - return !isClosed() ? baseSegment.asStorageAdapter() : null; - } - - @Override - public void close() - { - if (closed.compareAndSet(false, true)) { - referents.arriveAndDeregister(); - } else { - log.warn("close() is called more than once on ReferenceCountingSegment"); - } - } - - public ReferenceCounter referenceCounter() - { - return this; - } - - @Override - public boolean increment() - { - // Negative return from referents.register() means the Phaser is terminated. - return referents.register() >= 0; - } - - /** - * Returns a {@link Closeable} which action is to call {@link #decrement()} only once. If close() is called on the - * returned Closeable object for the second time, it won't call {@link #decrement()} again. - */ - @Override - public Closeable decrementOnceCloseable() - { - AtomicBoolean decremented = new AtomicBoolean(false); - return () -> { - if (decremented.compareAndSet(false, true)) { - decrement(); - } else { - log.warn("close() is called more than once on ReferenceCountingSegment.decrementOnceCloseable()"); - } - }; - } - - @Override - public void decrement() - { - referents.arriveAndDeregister(); - } - - @Override - public T as(Class clazz) - { - return getBaseSegment().as(clazz); + return !isClosed() ? baseObject.asStorageAdapter() : null; } @Override public boolean overshadows(ReferenceCountingSegment other) { - if (baseSegment.getId().getDataSource().equals(other.baseSegment.getId().getDataSource()) - && baseSegment.getId().getInterval().overlaps(other.baseSegment.getId().getInterval())) { - final int majorVersionCompare = baseSegment.getId().getVersion() - .compareTo(other.baseSegment.getId().getVersion()); + if (baseObject.getId().getDataSource().equals(other.baseObject.getId().getDataSource()) + && baseObject.getId().getInterval().overlaps(other.baseObject.getId().getInterval())) { + final int majorVersionCompare = baseObject.getId().getVersion().compareTo(other.baseObject.getId().getVersion()); if (majorVersionCompare > 0) { return true; } else if (majorVersionCompare == 0) { @@ -245,7 +152,7 @@ public int getEndRootPartitionId() @Override public String getVersion() { - return baseSegment.getId().getVersion(); + return baseObject.getId().getVersion(); } @Override @@ -259,4 +166,10 @@ public short getAtomicUpdateGroupSize() { return atomicUpdateGroupSize; } + + @Override + public Optional acquireReferences() + { + return incrementReferenceAndDecrementOnceCloseable(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java index 7b21d1f0186c..77e8978e2bee 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java @@ -96,7 +96,7 @@ static ColumnCapabilities getColumnCapabilities( { if (ColumnHolder.TIME_COLUMN_NAME.equals(columnName)) { // TIME_COLUMN_NAME is handled specially; override the provided rowSignature. - return new ColumnCapabilitiesImpl().setType(ValueType.LONG).setIsComplete(true); + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); } else { final ValueType valueType = rowSignature.getColumnType(columnName).orElse(null); @@ -105,12 +105,13 @@ static ColumnCapabilities getColumnCapabilities( // causes expression selectors to always treat us as arrays. If we might have multiple values (i.e. if our type // is nonnumeric), set isComplete false to compensate. if (valueType != null) { + if (valueType.isNumeric()) { + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(valueType); + } return new ColumnCapabilitiesImpl() .setType(valueType) .setDictionaryValuesUnique(false) - .setDictionaryValuesSorted(false) - // Numeric types should be reported as complete, but not STRING or COMPLEX (because we don't have full info) - .setIsComplete(valueType.isNumeric()); + .setDictionaryValuesSorted(false); } else { return null; } 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 512cbc22a0ce..36e84e8bd0e9 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedSegment.java @@ -30,7 +30,7 @@ /** * A {@link Segment} that is based on a stream of objects. */ -public class RowBasedSegment extends AbstractSegment +public class RowBasedSegment implements Segment { private final SegmentId segmentId; private final StorageAdapter storageAdapter; 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 8fb9b4e620dc..245c776b0aca 100644 --- a/processing/src/main/java/org/apache/druid/segment/Segment.java +++ b/processing/src/main/java/org/apache/druid/segment/Segment.java @@ -57,6 +57,15 @@ public interface Segment extends Closeable * @param desired interface * @return instance of clazz, or null if the interface is not supported by this segment */ + @SuppressWarnings("unused") @Nullable - T as(Class clazz); + default T as(Class clazz) + { + if (clazz.equals(QueryableIndex.class)) { + return (T) asQueryableIndex(); + } else if (clazz.equals(StorageAdapter.class)) { + return (T) asStorageAdapter(); + } + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/SegmentReference.java b/processing/src/main/java/org/apache/druid/segment/SegmentReference.java new file mode 100644 index 000000000000..fae2a7b36f65 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/SegmentReference.java @@ -0,0 +1,30 @@ +/* + * 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; + +/** + * A {@link Segment} with a associated references, such as {@link ReferenceCountingSegment} where the reference is + * the segment itself, and {@link org.apache.druid.segment.join.HashJoinSegment} which wraps a + * {@link ReferenceCountingSegment} and also includes the associated list of + * {@link org.apache.druid.segment.join.JoinableClause} + */ +public interface SegmentReference extends Segment, ReferenceCountedObject +{ +} 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 6fa16a6bcf26..e7905b2902e8 100644 --- a/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/StorageAdapter.java @@ -30,7 +30,7 @@ /** */ @PublicApi -public interface StorageAdapter extends CursorFactory +public interface StorageAdapter extends CursorFactory, ColumnInspector { Interval getInterval(); Indexed getAvailableDimensions(); @@ -62,6 +62,7 @@ public interface StorageAdapter extends CursorFactory * * @return capabilities, or null */ + @Override @Nullable ColumnCapabilities getColumnCapabilities(String column); diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index ada146d56d41..c0200e1e2eff 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -233,7 +233,8 @@ public String getValueFromSortedId(int index) private final DimensionDictionary dimLookup; private final MultiValueHandling multiValueHandling; private final boolean hasBitmapIndexes; - private boolean hasMultipleValues = false; + private volatile boolean hasMultipleValues = false; + private volatile boolean isSparse = false; @Nullable private SortedDimensionDictionary sortedLookup; @@ -301,6 +302,12 @@ public int[] processRowValsToUnsortedEncodedKeyComponent(@Nullable Object dimVal return encodedDimensionValues; } + @Override + public void setSparseIndexed() + { + isSparse = true; + } + @Override public long estimateEncodedKeyComponentSize(int[] key) { @@ -623,7 +630,9 @@ public String lookupName(int id) @Override public boolean nameLookupPossibleInAdvance() { - return true; + // name lookup is possible in advance if we got a value for every row (setSparseIndexed was not called on this + // column) or we've encountered an actual null value and it is present in our dictionary + return !isSparse || dimLookup.idForNull != ABSENT_VALUE_ID; } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java index cff7ef99f805..abb4637dc7b8 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionMergerV9.java @@ -221,7 +221,7 @@ protected void setupEncodedValueWriter() throws IOException final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); String filenameBase = StringUtils.format("%s.forward_dim", dimensionName); - if (capabilities.hasMultipleValues()) { + if (capabilities.hasMultipleValues().isTrue()) { if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) { encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create( dimensionName, @@ -533,7 +533,7 @@ public boolean canSkip() public ColumnDescriptor makeColumnDescriptor() { // Now write everything - boolean hasMultiValue = capabilities.hasMultipleValues(); + boolean hasMultiValue = capabilities.hasMultipleValues().isTrue(); final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression(); final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory(); diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java index 1b7163e0829b..cde454b70524 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnBuilder.java @@ -118,7 +118,6 @@ public ColumnHolder build() .setDictionaryValuesUnique(dictionaryEncoded) .setHasSpatialIndexes(spatialIndex != null) .setHasMultipleValues(hasMultipleValues) - .setIsComplete(true) .setFilterable(filterable), columnSupplier, bitmapIndex, diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java index 53f7440e87ef..a9af25b4602c 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilities.java @@ -19,33 +19,26 @@ package org.apache.druid.segment.column; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.druid.java.util.common.StringUtils; + +import javax.annotation.Nullable; + /** */ public interface ColumnCapabilities { ValueType getType(); - boolean isDictionaryEncoded(); Capable areDictionaryValuesSorted(); Capable areDictionaryValuesUnique(); boolean isRunLengthEncoded(); boolean hasBitmapIndexes(); boolean hasSpatialIndexes(); - boolean hasMultipleValues(); + Capable hasMultipleValues(); boolean isFilterable(); - /** - * This property indicates that this {@link ColumnCapabilities} is "complete" in that all properties can be expected - * to supply valid responses. This is mostly a hack to work around {@link ColumnCapabilities} generators that - * fail to set {@link #hasMultipleValues()} even when the associated column really could have multiple values. - * Until this situation is sorted out, if this method returns false, callers are encouraged to ignore - * {@link #hasMultipleValues()} and treat that property as if it were unknown. - * - * todo: replace all booleans with {@link Capable} and this method can be dropped - */ - boolean isComplete(); - - enum Capable { FALSE, @@ -57,6 +50,21 @@ public boolean isTrue() return this == TRUE; } + public boolean isMaybeTrue() + { + return isTrue() || isUnknown(); + } + + public boolean isUnknown() + { + return this == UNKNOWN; + } + + public Capable coerceUnknownToBoolean(boolean unknownIsTrue) + { + return this == UNKNOWN ? Capable.of(unknownIsTrue) : this; + } + public Capable and(Capable other) { if (this == UNKNOWN || other == UNKNOWN) { @@ -65,9 +73,36 @@ public Capable and(Capable other) return this == TRUE && other == TRUE ? TRUE : FALSE; } + public Capable or(Capable other) + { + if (this == TRUE) { + return TRUE; + } + return other; + } + public static Capable of(boolean bool) { return bool ? TRUE : FALSE; } + + @JsonCreator + public static Capable ofNullable(@Nullable Boolean bool) + { + return bool == null ? Capable.UNKNOWN : of(bool); + } + + @JsonValue + @Nullable + public Boolean toJson() + { + return this == UNKNOWN ? null : isTrue(); + } + + @Override + public String toString() + { + return StringUtils.toLowerCase(super.toString()); + } } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java index bee28ebdf832..9ddbd04a372f 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnCapabilitiesImpl.java @@ -31,15 +31,65 @@ */ public class ColumnCapabilitiesImpl implements ColumnCapabilities { - public static ColumnCapabilitiesImpl copyOf(final ColumnCapabilities other) + public static ColumnCapabilitiesImpl copyOf(@Nullable final ColumnCapabilities other) { final ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); - capabilities.merge(other); - capabilities.setFilterable(other.isFilterable()); - capabilities.setIsComplete(other.isComplete()); + if (other != null) { + capabilities.type = other.getType(); + capabilities.dictionaryEncoded = other.isDictionaryEncoded(); + capabilities.runLengthEncoded = other.isRunLengthEncoded(); + capabilities.hasInvertedIndexes = other.hasBitmapIndexes(); + capabilities.hasSpatialIndexes = other.hasSpatialIndexes(); + capabilities.hasMultipleValues = other.hasMultipleValues(); + capabilities.dictionaryValuesSorted = other.areDictionaryValuesSorted(); + capabilities.dictionaryValuesUnique = other.areDictionaryValuesUnique(); + capabilities.filterable = other.isFilterable(); + } return capabilities; } + /** + * Used at indexing time to finalize all {@link Capable#UNKNOWN} values to + * {@link Capable#FALSE}, in order to present a snapshot of the state of the this column + */ + @Nullable + public static ColumnCapabilitiesImpl snapshot(@Nullable final ColumnCapabilities capabilities) + { + return snapshot(capabilities, false); + } + + /** + * Used at indexing time to finalize all {@link Capable#UNKNOWN} values to + * {@link Capable#FALSE} or {@link Capable#TRUE}, in order to present a snapshot of the state of the this column + */ + @Nullable + public static ColumnCapabilitiesImpl snapshot(@Nullable final ColumnCapabilities capabilities, boolean unknownIsTrue) + { + if (capabilities == null) { + return null; + } + ColumnCapabilitiesImpl copy = copyOf(capabilities); + copy.hasMultipleValues = copy.hasMultipleValues.coerceUnknownToBoolean(unknownIsTrue); + copy.dictionaryValuesSorted = copy.dictionaryValuesSorted.coerceUnknownToBoolean(unknownIsTrue); + copy.dictionaryValuesUnique = copy.dictionaryValuesUnique.coerceUnknownToBoolean(unknownIsTrue); + return copy; + } + + + /** + * Create a no frills, simple column with {@link ValueType} set and everything else false + */ + public static ColumnCapabilitiesImpl createSimpleNumericColumnCapabilities(ValueType valueType) + { + return new ColumnCapabilitiesImpl().setType(valueType) + .setHasMultipleValues(false) + .setHasBitmapIndexes(false) + .setDictionaryEncoded(false) + .setDictionaryValuesSorted(false) + .setDictionaryValuesUnique(false) + .setHasSpatialIndexes(false); + } + @Nullable private ValueType type = null; @@ -47,7 +97,7 @@ public static ColumnCapabilitiesImpl copyOf(final ColumnCapabilities other) private boolean runLengthEncoded = false; private boolean hasInvertedIndexes = false; private boolean hasSpatialIndexes = false; - private boolean hasMultipleValues = false; + private Capable hasMultipleValues = Capable.UNKNOWN; // These capabilities are computed at query time and not persisted in the segment files. @JsonIgnore @@ -56,8 +106,6 @@ public static ColumnCapabilitiesImpl copyOf(final ColumnCapabilities other) private Capable dictionaryValuesUnique = Capable.UNKNOWN; @JsonIgnore private boolean filterable; - @JsonIgnore - private boolean complete = false; @Override @JsonProperty @@ -144,14 +192,14 @@ public ColumnCapabilitiesImpl setHasSpatialIndexes(boolean hasSpatialIndexes) @Override @JsonProperty("hasMultipleValues") - public boolean hasMultipleValues() + public Capable hasMultipleValues() { return hasMultipleValues; } public ColumnCapabilitiesImpl setHasMultipleValues(boolean hasMultipleValues) { - this.hasMultipleValues = hasMultipleValues; + this.hasMultipleValues = Capable.of(hasMultipleValues); return this; } @@ -171,22 +219,10 @@ public ColumnCapabilitiesImpl setFilterable(boolean filterable) return this; } - @Override - public boolean isComplete() - { - return complete; - } - - public ColumnCapabilitiesImpl setIsComplete(boolean complete) - { - this.complete = complete; - return this; - } - - public void merge(ColumnCapabilities other) + public ColumnCapabilities merge(@Nullable ColumnCapabilities other) { if (other == null) { - return; + return this; } if (type == null) { @@ -201,10 +237,11 @@ public void merge(ColumnCapabilities other) this.runLengthEncoded |= other.isRunLengthEncoded(); this.hasInvertedIndexes |= other.hasBitmapIndexes(); this.hasSpatialIndexes |= other.hasSpatialIndexes(); - this.hasMultipleValues |= other.hasMultipleValues(); - this.complete &= other.isComplete(); // these should always be the same? this.filterable &= other.isFilterable(); + this.hasMultipleValues = this.hasMultipleValues.or(other.hasMultipleValues()); this.dictionaryValuesSorted = this.dictionaryValuesSorted.and(other.areDictionaryValuesSorted()); this.dictionaryValuesUnique = this.dictionaryValuesUnique.and(other.areDictionaryValuesUnique()); + + return this; } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java index ef7613c6137c..0baa59490712 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java @@ -115,7 +115,7 @@ public boolean supportsBitmapIndex(final BitmapIndexSelector selector) // multiple values. The lack of multiple values is important because expression filters treat multi-value // arrays as nulls, which doesn't permit index based filtering. final String column = Iterables.getOnlyElement(requiredBindings.get()); - return selector.getBitmapIndex(column) != null && !selector.hasMultipleValues(column); + return selector.getBitmapIndex(column) != null && !selector.hasMultipleValues(column).isMaybeTrue(); } else { // Multi-column expression. return false; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/Filters.java b/processing/src/main/java/org/apache/druid/segment/filter/Filters.java index a1f536d22b3b..990127f99f04 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/Filters.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/Filters.java @@ -45,10 +45,12 @@ import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.filter.cnf.CalciteCnfHelper; import org.apache.druid.segment.filter.cnf.HiveCnfHelper; +import org.apache.druid.segment.join.filter.AllNullColumnSelectorFactory; import javax.annotation.Nullable; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; @@ -60,6 +62,7 @@ */ public class Filters { + private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory(); /** * Convert a list of DimFilters to a list of Filters. @@ -411,7 +414,7 @@ static boolean supportsSelectivityEstimation( if (filter.supportsBitmapIndex(indexSelector)) { final ColumnHolder columnHolder = columnSelector.getColumnHolder(dimension); if (columnHolder != null) { - return !columnHolder.getCapabilities().hasMultipleValues(); + return !columnHolder.getCapabilities().hasMultipleValues().isMaybeTrue(); } } return false; @@ -521,4 +524,32 @@ public static Filter or(Set filterSet) return new OrFilter(filterSet); } + + /** + * @param filter the filter. + * @return The normalized or clauses for the provided filter. + */ + public static Set toNormalizedOrClauses(Filter filter) + { + Filter normalizedFilter = Filters.toCnf(filter); + + // List of candidates for pushdown + // CNF normalization will generate either + // - an AND filter with multiple subfilters + // - or a single non-AND subfilter which cannot be split further + Set normalizedOrClauses; + if (normalizedFilter instanceof AndFilter) { + normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters(); + } else { + normalizedOrClauses = Collections.singleton(normalizedFilter); + } + return normalizedOrClauses; + } + + + public static boolean filterMatchesNull(Filter filter) + { + ValueMatcher valueMatcher = filter.makeMatcher(ALL_NULL_COLUMN_SELECTOR_FACTORY); + return valueMatcher.matches(); + } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java b/processing/src/main/java/org/apache/druid/segment/generator/ColumnValueGenerator.java similarity index 90% rename from benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java rename to processing/src/main/java/org/apache/druid/segment/generator/ColumnValueGenerator.java index 5fe89d8a9fb5..fbfc1a9f3c15 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkColumnValueGenerator.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/ColumnValueGenerator.java @@ -17,12 +17,13 @@ * under the License. */ -package org.apache.druid.benchmark.datagen; +package org.apache.druid.segment.generator; import org.apache.commons.math3.distribution.AbstractIntegerDistribution; import org.apache.commons.math3.distribution.AbstractRealDistribution; import org.apache.commons.math3.distribution.EnumeratedDistribution; import org.apache.commons.math3.distribution.NormalDistribution; +import org.apache.commons.math3.distribution.UniformIntegerDistribution; import org.apache.commons.math3.distribution.UniformRealDistribution; import org.apache.commons.math3.distribution.ZipfDistribution; import org.apache.commons.math3.util.Pair; @@ -33,16 +34,16 @@ import java.util.List; import java.util.Random; -public class BenchmarkColumnValueGenerator +public class ColumnValueGenerator { - private final BenchmarkColumnSchema schema; + private final GeneratorColumnSchema schema; private final long seed; private Serializable distribution; private Random simpleRng; - public BenchmarkColumnValueGenerator( - BenchmarkColumnSchema schema, + public ColumnValueGenerator( + GeneratorColumnSchema schema, long seed ) { @@ -76,7 +77,7 @@ public Object generateRowValue() } } - public BenchmarkColumnSchema getSchema() + public GeneratorColumnSchema getSchema() { return schema; } @@ -138,7 +139,7 @@ private Object convertType(Object input, ValueType type) private void initDistribution() { - BenchmarkColumnSchema.ValueDistribution distributionType = schema.getDistributionType(); + GeneratorColumnSchema.ValueDistribution distributionType = schema.getDistributionType(); ValueType type = schema.getType(); List enumeratedValues = schema.getEnumeratedValues(); List enumeratedProbabilities = schema.getEnumeratedProbabilities(); @@ -195,6 +196,15 @@ private void initDistribution() } distribution = new EnumeratedTreeDistribution<>(probabilities); break; + case LAZY_ZIPF: + int lazyCardinality; + Integer startInt = schema.getStartInt(); + lazyCardinality = schema.getEndInt() - startInt; + distribution = new ZipfDistribution(lazyCardinality, schema.getZipfExponent()); + break; + case LAZY_DISCRETE_UNIFORM: + distribution = new UniformIntegerDistribution(schema.getStartInt(), schema.getEndInt()); + break; case ENUMERATED: for (int i = 0; i < enumeratedValues.size(); i++) { probabilities.add(new Pair<>(enumeratedValues.get(i), enumeratedProbabilities.get(i))); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkDataGenerator.java b/processing/src/main/java/org/apache/druid/segment/generator/DataGenerator.java similarity index 83% rename from benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkDataGenerator.java rename to processing/src/main/java/org/apache/druid/segment/generator/DataGenerator.java index 49d01a4c12c5..f2fac21d536e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkDataGenerator.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/DataGenerator.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.benchmark.datagen; +package org.apache.druid.segment.generator; import com.google.common.base.Function; import com.google.common.base.Preconditions; @@ -31,12 +31,12 @@ import java.util.List; import java.util.Map; -public class BenchmarkDataGenerator +public class DataGenerator { - private final List columnSchemas; + private final List columnSchemas; private final long seed; - private List columnGenerators; + private List columnGenerators; private final long startTime; private final long endTime; private final int numConsecutiveTimestamps; @@ -46,8 +46,8 @@ public class BenchmarkDataGenerator private int timeCounter; private List dimensionNames; - public BenchmarkDataGenerator( - List columnSchemas, + public DataGenerator( + List columnSchemas, final long seed, long startTime, int numConsecutiveTimestamps, @@ -66,8 +66,8 @@ public BenchmarkDataGenerator( init(); } - public BenchmarkDataGenerator( - List columnSchemas, + public DataGenerator( + List columnSchemas, final long seed, Interval interval, int numRows @@ -91,7 +91,7 @@ public BenchmarkDataGenerator( public InputRow nextRow() { Map event = new HashMap<>(); - for (BenchmarkColumnValueGenerator generator : columnGenerators) { + for (ColumnValueGenerator generator : columnGenerators) { event.put(generator.getSchema().getName(), generator.generateRowValue()); } MapBasedInputRow row = new MapBasedInputRow(nextTimestamp(), dimensionNames, event); @@ -104,7 +104,7 @@ private void init() this.currentTime = startTime; dimensionNames = new ArrayList<>(); - for (BenchmarkColumnSchema schema : columnSchemas) { + for (GeneratorColumnSchema schema : columnSchemas) { if (!schema.isMetric()) { dimensionNames.add(schema.getName()); } @@ -114,11 +114,11 @@ private void init() columnGenerators.addAll( Lists.transform( columnSchemas, - new Function() + new Function() { @Override - public BenchmarkColumnValueGenerator apply( - BenchmarkColumnSchema input + public ColumnValueGenerator apply( + GeneratorColumnSchema input ) { return input.makeGenerator(seed); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/EnumeratedTreeDistribution.java b/processing/src/main/java/org/apache/druid/segment/generator/EnumeratedTreeDistribution.java similarity index 95% rename from benchmarks/src/test/java/org/apache/druid/benchmark/datagen/EnumeratedTreeDistribution.java rename to processing/src/main/java/org/apache/druid/segment/generator/EnumeratedTreeDistribution.java index 1b44e5c3216e..057af0775cfc 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/EnumeratedTreeDistribution.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/EnumeratedTreeDistribution.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.benchmark.datagen; +package org.apache.druid.segment.generator; import org.apache.commons.math3.distribution.EnumeratedDistribution; import org.apache.commons.math3.util.Pair; @@ -41,7 +41,7 @@ public EnumeratedTreeDistribution(final List> pmf) super(pmf); // build the interval tree - probabilityRanges = new TreeMap(); + probabilityRanges = new TreeMap<>(); normalizedPmf = this.getPmf(); double cumulativep = 0.0; for (int i = 0; i < normalizedPmf.size(); i++) { diff --git a/processing/src/main/java/org/apache/druid/segment/generator/GeneratorBasicSchemas.java b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorBasicSchemas.java new file mode 100644 index 000000000000..a7683963e68c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorBasicSchemas.java @@ -0,0 +1,420 @@ +/* + * 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.generator; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.java.util.common.Intervals; +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.aggregation.DoubleMinAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongMaxAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; +import org.apache.druid.segment.column.ValueType; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; + +public class GeneratorBasicSchemas +{ + private static final ImmutableMap.Builder SCHEMA_INFO_BUILDER = ImmutableMap.builder(); + + static { + // basic schema + List basicSchemaColumns = ImmutableList.of( + // dims + GeneratorColumnSchema.makeSequential("dimSequential", ValueType.STRING, false, 1, null, 0, 1000), + GeneratorColumnSchema.makeZipf("dimZipf", ValueType.STRING, false, 1, null, 1, 101, 1.0), + GeneratorColumnSchema.makeDiscreteUniform("dimUniform", ValueType.STRING, false, 1, null, 1, 100000), + GeneratorColumnSchema.makeSequential("dimSequentialHalfNull", ValueType.STRING, false, 1, 0.5, 0, 1000), + GeneratorColumnSchema.makeEnumerated( + "dimMultivalEnumerated", + ValueType.STRING, + false, + 4, + null, + Arrays.asList("Hello", "World", "Foo", "Bar", "Baz"), + Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) + ), + GeneratorColumnSchema.makeEnumerated( + "dimMultivalEnumerated2", + ValueType.STRING, + false, + 3, + null, + Arrays.asList("Apple", "Orange", "Xylophone", "Corundum", null), + Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) + ), + GeneratorColumnSchema.makeSequential("dimMultivalSequentialWithNulls", ValueType.STRING, false, 8, 0.15, 1, 11), + GeneratorColumnSchema.makeSequential("dimHyperUnique", ValueType.STRING, false, 1, null, 0, 100000), + GeneratorColumnSchema.makeSequential("dimNull", ValueType.STRING, false, 1, 1.0, 0, 1), + + // metrics + GeneratorColumnSchema.makeSequential("metLongSequential", ValueType.LONG, true, 1, null, 0, 10000), + GeneratorColumnSchema.makeDiscreteUniform("metLongUniform", ValueType.LONG, true, 1, null, 0, 500), + GeneratorColumnSchema.makeNormal("metFloatNormal", ValueType.FLOAT, true, 1, null, 5000.0, 1.0, true), + GeneratorColumnSchema.makeZipf("metFloatZipf", ValueType.FLOAT, true, 1, null, 0, 1000, 1.0) + ); + + List basicSchemaIngestAggs = new ArrayList<>(); + basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + basicSchemaIngestAggs.add(new LongSumAggregatorFactory("sumLongSequential", "metLongSequential")); + basicSchemaIngestAggs.add(new LongMaxAggregatorFactory("maxLongUniform", "metLongUniform")); + basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("sumFloatNormal", "metFloatNormal")); + basicSchemaIngestAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf")); + basicSchemaIngestAggs.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique")); + + List basicSchemaIngestAggsExpression = new ArrayList<>(); + basicSchemaIngestAggsExpression.add(new CountAggregatorFactory("rows")); + basicSchemaIngestAggsExpression.add(new LongSumAggregatorFactory( + "sumLongSequential", + null, + "if(sumLongSequential>0 && dimSequential>100 || dimSequential<10 || metLongSequential>3000,sumLongSequential,0)", + ExprMacroTable.nil() + )); + basicSchemaIngestAggsExpression.add(new LongMaxAggregatorFactory("maxLongUniform", "metLongUniform")); + basicSchemaIngestAggsExpression.add(new DoubleSumAggregatorFactory( + "sumFloatNormal", + null, + "if(sumFloatNormal>0 && dimSequential>100 || dimSequential<10 || metLongSequential>3000,sumFloatNormal,0)", + ExprMacroTable.nil() + )); + basicSchemaIngestAggsExpression.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf")); + basicSchemaIngestAggsExpression.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique")); + + Interval basicSchemaDataInterval = Intervals.of("2000-01-01/P1D"); + + GeneratorSchemaInfo basicSchema = new GeneratorSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggs, + basicSchemaDataInterval, + true + ); + + GeneratorSchemaInfo basicSchemaExpression = new GeneratorSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggsExpression, + basicSchemaDataInterval, + true + ); + + SCHEMA_INFO_BUILDER.put("basic", basicSchema); + SCHEMA_INFO_BUILDER.put("expression", basicSchemaExpression); + } + + static { // simple single string column and count agg schema, no rollup + List basicSchemaColumns = ImmutableList.of( + // dims + GeneratorColumnSchema.makeSequential("dimSequential", ValueType.STRING, false, 1, null, 0, 1000000) + ); + + List basicSchemaIngestAggs = new ArrayList<>(); + basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); + + GeneratorSchemaInfo basicSchema = new GeneratorSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggs, + basicSchemaDataInterval, + false + ); + SCHEMA_INFO_BUILDER.put("simple", basicSchema); + } + + static { + // simple single long column and count agg schema, no rollup + List basicSchemaColumns = ImmutableList.of( + // dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported + GeneratorColumnSchema.makeSequential("dimSequential", ValueType.LONG, true, 1, null, 0, 1000000) + ); + + List basicSchemaIngestAggs = new ArrayList<>(); + basicSchemaIngestAggs.add(new LongSumAggregatorFactory("dimSequential", "dimSequential")); + basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); + + GeneratorSchemaInfo basicSchema = new GeneratorSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggs, + basicSchemaDataInterval, + false + ); + SCHEMA_INFO_BUILDER.put("simpleLong", basicSchema); + } + + static { + // simple single float column and count agg schema, no rollup + List basicSchemaColumns = ImmutableList.of( + // dims, ingest as a metric for now with rollup off, until numeric dims at ingestion are supported + GeneratorColumnSchema.makeSequential("dimSequential", ValueType.FLOAT, true, 1, null, 0, 1000000) + ); + + List basicSchemaIngestAggs = new ArrayList<>(); + basicSchemaIngestAggs.add(new DoubleSumAggregatorFactory("dimSequential", "dimSequential")); + basicSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); + + GeneratorSchemaInfo basicSchema = new GeneratorSchemaInfo( + basicSchemaColumns, + basicSchemaIngestAggs, + basicSchemaDataInterval, + false + ); + SCHEMA_INFO_BUILDER.put("simpleFloat", basicSchema); + } + + static { + // schema with high opportunity for rollup + List rolloColumns = ImmutableList.of( + // dims + GeneratorColumnSchema.makeEnumerated( + "dimEnumerated", + ValueType.STRING, + false, + 1, + null, + Arrays.asList("Hello", "World", "Foo", "Bar", "Baz"), + Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) + ), + GeneratorColumnSchema.makeEnumerated( + "dimEnumerated2", + ValueType.STRING, + false, + 1, + null, + Arrays.asList("Apple", "Orange", "Xylophone", "Corundum", null), + Arrays.asList(0.2, 0.25, 0.15, 0.10, 0.3) + ), + GeneratorColumnSchema.makeZipf("dimZipf", ValueType.STRING, false, 1, null, 1, 100, 2.0), + GeneratorColumnSchema.makeDiscreteUniform("dimUniform", ValueType.STRING, false, 1, null, 1, 100), + + // metrics + GeneratorColumnSchema.makeZipf("metLongZipf", ValueType.LONG, true, 1, null, 0, 10000, 2.0), + GeneratorColumnSchema.makeDiscreteUniform("metLongUniform", ValueType.LONG, true, 1, null, 0, 500), + GeneratorColumnSchema.makeNormal("metFloatNormal", ValueType.FLOAT, true, 1, null, 5000.0, 1.0, true), + GeneratorColumnSchema.makeZipf("metFloatZipf", ValueType.FLOAT, true, 1, null, 0, 1000, 1.5) + ); + + List rolloSchemaIngestAggs = new ArrayList<>(); + rolloSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + rolloSchemaIngestAggs.add(new LongSumAggregatorFactory("sumLongSequential", "metLongSequential")); + rolloSchemaIngestAggs.add(new LongMaxAggregatorFactory("maxLongUniform", "metLongUniform")); + rolloSchemaIngestAggs.add(new DoubleSumAggregatorFactory("sumFloatNormal", "metFloatNormal")); + rolloSchemaIngestAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "metFloatZipf")); + rolloSchemaIngestAggs.add(new HyperUniquesAggregatorFactory("hyper", "dimHyperUnique")); + + Interval basicSchemaDataInterval = Intervals.utc(0, 1000000); + + GeneratorSchemaInfo rolloSchema = new GeneratorSchemaInfo( + rolloColumns, + rolloSchemaIngestAggs, + basicSchemaDataInterval, + true + ); + SCHEMA_INFO_BUILDER.put("rollo", rolloSchema); + } + + static { + // simple schema with null valued rows, no aggs on numeric columns + List nullsSchemaColumns = ImmutableList.of( + // string dims with nulls + GeneratorColumnSchema.makeZipf("stringZipf", ValueType.STRING, false, 1, 0.8, 1, 101, 1.5), + GeneratorColumnSchema.makeDiscreteUniform("stringUniform", ValueType.STRING, false, 1, 0.3, 1, 100000), + GeneratorColumnSchema.makeSequential("stringSequential", ValueType.STRING, false, 1, 0.5, 0, 1000), + + // numeric dims with nulls + GeneratorColumnSchema.makeSequential("longSequential", ValueType.LONG, false, 1, 0.45, 0, 10000), + GeneratorColumnSchema.makeDiscreteUniform("longUniform", ValueType.LONG, false, 1, 0.25, 0, 500), + GeneratorColumnSchema.makeZipf("doubleZipf", ValueType.DOUBLE, false, 1, 0.1, 0, 1000, 2.0), + GeneratorColumnSchema.makeZipf("floatZipf", ValueType.FLOAT, false, 1, 0.1, 0, 1000, 2.0) + ); + + List simpleNullsSchemaIngestAggs = new ArrayList<>(); + simpleNullsSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval nullsSchemaDataInterval = Intervals.of("2000-01-01/P1D"); + + GeneratorSchemaInfo nullsSchema = new GeneratorSchemaInfo( + nullsSchemaColumns, + simpleNullsSchemaIngestAggs, + nullsSchemaDataInterval, + false + ); + + SCHEMA_INFO_BUILDER.put("nulls", nullsSchema); + } + + static { + // simple schema with null valued rows, no aggs on numeric columns + List nullsSchemaColumns = ImmutableList.of( + // string dims + GeneratorColumnSchema.makeZipf("stringZipf", ValueType.STRING, false, 1, null, 1, 101, 1.5), + GeneratorColumnSchema.makeDiscreteUniform("stringUniform", ValueType.STRING, false, 1, null, 1, 100000), + GeneratorColumnSchema.makeSequential("stringSequential", ValueType.STRING, false, 1, null, 0, 1000), + + // string dims with nulls + GeneratorColumnSchema.makeZipf("stringZipfWithNulls", ValueType.STRING, false, 1, 0.8, 1, 101, 1.5), + GeneratorColumnSchema.makeDiscreteUniform("stringUniformWithNulls", ValueType.STRING, false, 1, 0.3, 1, 100000), + GeneratorColumnSchema.makeSequential("stringSequentialWithNulls", ValueType.STRING, false, 1, 0.5, 0, 1000), + + // numeric dims + GeneratorColumnSchema.makeSequential("longSequential", ValueType.LONG, false, 1, null, 0, 10000), + GeneratorColumnSchema.makeDiscreteUniform("longUniform", ValueType.LONG, false, 1, null, 0, 500), + GeneratorColumnSchema.makeZipf("doubleZipf", ValueType.DOUBLE, false, 1, null, 0, 1000, 2.0), + GeneratorColumnSchema.makeZipf("floatZipf", ValueType.FLOAT, false, 1, null, 0, 1000, 2.0), + + // numeric dims with nulls + GeneratorColumnSchema.makeSequential("longSequentialWithNulls", ValueType.LONG, false, 1, 0.45, 0, 10000), + GeneratorColumnSchema.makeDiscreteUniform("longUniformWithNulls", ValueType.LONG, false, 1, 0.25, 0, 500), + GeneratorColumnSchema.makeZipf("doubleZipfWithNulls", ValueType.DOUBLE, false, 1, 0.1, 0, 1000, 2.0), + GeneratorColumnSchema.makeZipf("floatZipfWithNulls", ValueType.FLOAT, false, 1, 0.1, 0, 1000, 2.0) + ); + + List simpleNullsSchemaIngestAggs = new ArrayList<>(); + simpleNullsSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval nullsSchemaDataInterval = Intervals.of("2000-01-01/P1D"); + + GeneratorSchemaInfo nullsSchema = new GeneratorSchemaInfo( + nullsSchemaColumns, + simpleNullsSchemaIngestAggs, + nullsSchemaDataInterval, + false + ); + + SCHEMA_INFO_BUILDER.put("nulls-and-non-nulls", nullsSchema); + } + + static { + // simple 'wide' schema with null valued rows, high cardinality columns, no aggs on numeric columns + // essentially 'nulls-and-non-nulls' with a ton of extra zipf columns of each type with a variety of value + // distributions + List nullsSchemaColumns = ImmutableList.of( + // string dims + GeneratorColumnSchema.makeLazyZipf("stringZipf", ValueType.STRING, false, 1, null, 1, 101, 1.5), + GeneratorColumnSchema.makeLazyDiscreteUniform("stringUniform", ValueType.STRING, false, 1, null, 1, 100000), + GeneratorColumnSchema.makeSequential("stringSequential", ValueType.STRING, false, 1, null, 0, 1000), + + // string dims with nulls + GeneratorColumnSchema.makeLazyZipf("stringZipfWithNulls", ValueType.STRING, false, 1, 0.8, 1, 101, 1.5), + GeneratorColumnSchema.makeLazyDiscreteUniform( + "stringUniformWithNulls", + ValueType.STRING, + false, + 1, + 0.3, + 1, + 100000 + ), + GeneratorColumnSchema.makeSequential("stringSequentialWithNulls", ValueType.STRING, false, 1, 0.5, 0, 1000), + + // additional string columns + GeneratorColumnSchema.makeLazyZipf("string1", ValueType.STRING, false, 1, 0.1, 1, 100000, 2.0), + GeneratorColumnSchema.makeLazyZipf("string2", ValueType.STRING, false, 1, 0.3, 1, 1000000, 1.5), + GeneratorColumnSchema.makeLazyZipf("string3", ValueType.STRING, false, 1, 0.12, 1, 1000, 1.25), + GeneratorColumnSchema.makeLazyZipf("string4", ValueType.STRING, false, 1, 0.22, 1, 12000, 3.0), + GeneratorColumnSchema.makeLazyZipf("string5", ValueType.STRING, false, 1, 0.05, 1, 33333, 1.8), + GeneratorColumnSchema.makeLazyZipf("string6", ValueType.STRING, false, 1, 0.09, 1, 25000000, 2.0), + GeneratorColumnSchema.makeLazyZipf("string7", ValueType.STRING, false, 1, 0.8, 1, 100, 1.5), + GeneratorColumnSchema.makeLazyZipf("string8", ValueType.STRING, false, 1, 0.5, 1, 10, 1.2), + GeneratorColumnSchema.makeLazyZipf("string9", ValueType.STRING, false, 1, 0.05, 1, 1000000, 1.3), + GeneratorColumnSchema.makeLazyZipf("string10", ValueType.STRING, false, 1, 0.4, 1, 888888, 1.4), + GeneratorColumnSchema.makeLazyZipf("string11", ValueType.STRING, false, 1, 0.7, 1, 999, 1.8), + GeneratorColumnSchema.makeLazyZipf("string12", ValueType.STRING, false, 1, 0.2, 1, 123456, 1.0), + GeneratorColumnSchema.makeLazyZipf("string13", ValueType.STRING, false, 1, 0.33, 1, 52, 1.9), + GeneratorColumnSchema.makeLazyZipf("string14", ValueType.STRING, false, 1, 0.42, 1, 90001, 1.75), + + // numeric dims + GeneratorColumnSchema.makeSequential("longSequential", ValueType.LONG, false, 1, null, 0, 10000), + GeneratorColumnSchema.makeLazyDiscreteUniform("longUniform", ValueType.LONG, false, 1, null, 0, 500), + + GeneratorColumnSchema.makeLazyZipf("longZipf", ValueType.LONG, false, 1, null, 0, 1000, 2.0), + GeneratorColumnSchema.makeLazyZipf("doubleZipf", ValueType.DOUBLE, false, 1, null, 0, 1000, 2.0), + GeneratorColumnSchema.makeLazyZipf("floatZipf", ValueType.FLOAT, false, 1, null, 0, 1000, 2.0), + + // numeric dims with nulls + GeneratorColumnSchema.makeSequential("longSequentialWithNulls", ValueType.LONG, false, 1, 0.45, 0, 10000), + GeneratorColumnSchema.makeLazyDiscreteUniform("longUniformWithNulls", ValueType.LONG, false, 1, 0.25, 0, 500), + + GeneratorColumnSchema.makeLazyZipf("longZipfWithNulls", ValueType.LONG, false, 1, 0.1, 0, 1000, 2.0), + GeneratorColumnSchema.makeLazyZipf("doubleZipfWithNulls", ValueType.DOUBLE, false, 1, 0.1, 0, 1000, 2.0), + GeneratorColumnSchema.makeLazyZipf("floatZipfWithNulls", ValueType.FLOAT, false, 1, 0.1, 0, 1000, 2.0), + + // additional numeric columns + GeneratorColumnSchema.makeLazyZipf("long1", ValueType.LONG, false, 1, 0.1, 0, 1001, 2.0), + GeneratorColumnSchema.makeLazyZipf("long2", ValueType.LONG, false, 1, 0.01, 0, 666666, 2.2), + GeneratorColumnSchema.makeLazyZipf("long3", ValueType.LONG, false, 1, 0.12, 0, 1000000, 2.5), + GeneratorColumnSchema.makeLazyZipf("long4", ValueType.LONG, false, 1, 0.4, 0, 23, 1.2), + GeneratorColumnSchema.makeLazyZipf("long5", ValueType.LONG, false, 1, 0.33, 0, 9999, 1.5), + GeneratorColumnSchema.makeLazyZipf("long6", ValueType.LONG, false, 1, 0.8, 0, 12345, 1.8), + GeneratorColumnSchema.makeLazyZipf("long7", ValueType.LONG, false, 1, 0.6, 0, 543210, 2.3), + GeneratorColumnSchema.makeLazyZipf("long8", ValueType.LONG, false, 1, 0.5, 0, 11223344, 2.4), + GeneratorColumnSchema.makeLazyZipf("long9", ValueType.LONG, false, 1, 0.111, 0, 80, 4.0), + GeneratorColumnSchema.makeLazyZipf("long10", ValueType.LONG, false, 1, 0.21, 0, 111, 3.3), + + GeneratorColumnSchema.makeLazyZipf("double1", ValueType.DOUBLE, false, 1, 0.1, 0, 333, 2.2), + GeneratorColumnSchema.makeLazyZipf("double2", ValueType.DOUBLE, false, 1, 0.01, 0, 4021, 2.5), + GeneratorColumnSchema.makeLazyZipf("double3", ValueType.DOUBLE, false, 1, 0.41, 0, 90210, 4.0), + GeneratorColumnSchema.makeLazyZipf("double4", ValueType.DOUBLE, false, 1, 0.5, 0, 5555555, 1.2), + GeneratorColumnSchema.makeLazyZipf("double5", ValueType.DOUBLE, false, 1, 0.23, 0, 80, 1.8), + GeneratorColumnSchema.makeLazyZipf("double6", ValueType.DOUBLE, false, 1, 0.33, 0, 99999, 3.0), + GeneratorColumnSchema.makeLazyZipf("double7", ValueType.DOUBLE, false, 1, 0.8, 0, 12345678, 1.4), + GeneratorColumnSchema.makeLazyZipf("double8", ValueType.DOUBLE, false, 1, 0.4, 0, 8080, 4.2), + GeneratorColumnSchema.makeLazyZipf("double9", ValueType.DOUBLE, false, 1, 0.111, 0, 9999, 2.3), + GeneratorColumnSchema.makeLazyZipf("double10", ValueType.DOUBLE, false, 1, 0.2, 0, 19, 0.6), + + GeneratorColumnSchema.makeLazyZipf("float1", ValueType.FLOAT, false, 1, 0.11, 0, 1000000, 1.7), + GeneratorColumnSchema.makeLazyZipf("float2", ValueType.FLOAT, false, 1, 0.4, 0, 10, 1.5), + GeneratorColumnSchema.makeLazyZipf("float3", ValueType.FLOAT, false, 1, 0.8, 0, 5000, 2.3), + GeneratorColumnSchema.makeLazyZipf("float4", ValueType.FLOAT, false, 1, 0.999, 0, 14440, 2.0), + GeneratorColumnSchema.makeLazyZipf("float5", ValueType.FLOAT, false, 1, 0.001, 0, 1029, 1.5), + GeneratorColumnSchema.makeLazyZipf("float6", ValueType.FLOAT, false, 1, 0.01, 0, 4445555, 1.8), + GeneratorColumnSchema.makeLazyZipf("float7", ValueType.FLOAT, false, 1, 0.44, 0, 1000000, 1.1), + GeneratorColumnSchema.makeLazyZipf("float8", ValueType.FLOAT, false, 1, 0.55, 0, 33, 4.5), + GeneratorColumnSchema.makeLazyZipf("float9", ValueType.FLOAT, false, 1, 0.12, 0, 5000, 2.2), + GeneratorColumnSchema.makeLazyZipf("float10", ValueType.FLOAT, false, 1, 0.11, 0, 101, 1.3) + ); + + List simpleNullsSchemaIngestAggs = new ArrayList<>(); + simpleNullsSchemaIngestAggs.add(new CountAggregatorFactory("rows")); + + Interval nullsSchemaDataInterval = Intervals.of("2000-01-01/P1D"); + + GeneratorSchemaInfo nullsSchema = new GeneratorSchemaInfo( + nullsSchemaColumns, + simpleNullsSchemaIngestAggs, + nullsSchemaDataInterval, + false + ); + + SCHEMA_INFO_BUILDER.put("wide", nullsSchema); + } + + public static final Map SCHEMA_MAP = SCHEMA_INFO_BUILDER.build(); +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkColumnSchema.java b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorColumnSchema.java similarity index 58% rename from benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkColumnSchema.java rename to processing/src/main/java/org/apache/druid/segment/generator/GeneratorColumnSchema.java index 8636941dd28d..2c5e42d6651d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkColumnSchema.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorColumnSchema.java @@ -17,8 +17,11 @@ * under the License. */ -package org.apache.druid.benchmark.datagen; +package org.apache.druid.segment.generator; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DoubleDimensionSchema; import org.apache.druid.data.input.impl.FloatDimensionSchema; @@ -28,26 +31,32 @@ import org.apache.druid.segment.column.ValueType; import java.util.List; +import java.util.Objects; -public class BenchmarkColumnSchema +public class GeneratorColumnSchema { + /** - * SEQUENTIAL: Generate integer or enumerated values in sequence. Not random. + * SEQUENTIAL: Generate integer or enumerated values in sequence. Not random. + * + * DISCRETE_UNIFORM: Discrete uniform distribution, generates integers or enumerated values. + * + * ROUNDED_NORMAL: Discrete distribution that rounds sample values from an underlying normal + * distribution * - * DISCRETE_UNIFORM: Discrete uniform distribution, generates integers or enumerated values. + * ZIPF: Discrete Zipf distribution. + * Lower numbers have higher probability. + * Can also generate Zipf distribution from a list of enumerated values. * - * ROUNDED_NORMAL: Discrete distribution that rounds sample values from an underlying normal - * distribution + * LAZY_ZIPF: ZIPF but lazy evaluated for large cardinalities * - * ZIPF: Discrete Zipf distribution. - * Lower numbers have higher probability. - * Can also generate Zipf distribution from a list of enumerated values. + * LAZY_DISCRETE_UNIFORM: DISCRETE_UNIFORM but lazy evaluated for large cardinalities * - * ENUMERATED: Discrete distribution, generated from lists of values and associated probabilities. + * ENUMERATED: Discrete distribution, generated from lists of values and associated probabilities. * - * NORMAL: Continuous normal distribution. + * NORMAL: Continuous normal distribution. * - * UNIFORM: Continuous uniform distribution. + * UNIFORM: Continuous uniform distribution. */ public enum ValueDistribution { @@ -57,6 +66,8 @@ public enum ValueDistribution ROUNDED_NORMAL, ZIPF, ENUMERATED, + LAZY_ZIPF, + LAZY_DISCRETE_UNIFORM, // continuous distributions UNIFORM, @@ -128,7 +139,43 @@ public enum ValueDistribution private Double mean; private Double standardDeviation; - private BenchmarkColumnSchema( + @JsonCreator + public GeneratorColumnSchema( + @JsonProperty("name") String name, + @JsonProperty("type") ValueType type, + @JsonProperty("isMetric") boolean isMetric, + @JsonProperty("rowSize") int rowSize, + @JsonProperty("nullProbability") Double nullProbability, + @JsonProperty("distributionType") ValueDistribution distributionType, + @JsonProperty("enumeratedValues") List enumeratedValues, + @JsonProperty("enumeratedProbabilities") List enumeratedProbabilities, + @JsonProperty("startInt") Integer startInt, + @JsonProperty("endInt") Integer endInt, + @JsonProperty("startDouble") Double startDouble, + @JsonProperty("endDouble") Double endDouble, + @JsonProperty("zipfExponent") Double zipfExponent, + @JsonProperty("mean") Double mean, + @JsonProperty("standardDeviation") Double standardDeviation + ) + { + this.name = name; + this.type = type; + this.isMetric = isMetric; + this.distributionType = distributionType; + this.rowSize = rowSize; + this.nullProbability = nullProbability; + this.enumeratedValues = enumeratedValues; + this.enumeratedProbabilities = enumeratedProbabilities; + this.startInt = startInt; + this.endInt = endInt; + this.startDouble = startDouble; + this.endDouble = endDouble; + this.zipfExponent = zipfExponent; + this.mean = mean; + this.standardDeviation = standardDeviation; + } + + private GeneratorColumnSchema( String name, ValueType type, boolean isMetric, @@ -145,11 +192,12 @@ private BenchmarkColumnSchema( this.nullProbability = nullProbability; } - public BenchmarkColumnValueGenerator makeGenerator(long seed) + public ColumnValueGenerator makeGenerator(long seed) { - return new BenchmarkColumnValueGenerator(this, seed); + return new ColumnValueGenerator(this, seed); } + @JsonIgnore public DimensionSchema getDimensionSchema() { switch (type) { @@ -166,82 +214,97 @@ public DimensionSchema getDimensionSchema() } } + @JsonProperty public String getName() { return name; } + @JsonProperty public Double getNullProbability() { return nullProbability; } + @JsonProperty public ValueType getType() { return type; } + @JsonProperty public boolean isMetric() { return isMetric; } + @JsonProperty public ValueDistribution getDistributionType() { return distributionType; } + @JsonProperty public int getRowSize() { return rowSize; } + @JsonProperty public List getEnumeratedValues() { return enumeratedValues; } + @JsonProperty public List getEnumeratedProbabilities() { return enumeratedProbabilities; } + @JsonProperty public Integer getStartInt() { return startInt; } + @JsonProperty public Integer getEndInt() { return endInt; } + @JsonProperty public Double getStartDouble() { return startDouble; } + @JsonProperty public Double getEndDouble() { return endDouble; } + @JsonProperty public Double getZipfExponent() { return zipfExponent; } + @JsonProperty public Double getMean() { return mean; } + @JsonProperty public Double getStandardDeviation() { return standardDeviation; } - public static BenchmarkColumnSchema makeSequential( + public static GeneratorColumnSchema makeSequential( String name, ValueType type, boolean isMetric, @@ -251,7 +314,7 @@ public static BenchmarkColumnSchema makeSequential( int endInt ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -264,7 +327,7 @@ public static BenchmarkColumnSchema makeSequential( return schema; } - public static BenchmarkColumnSchema makeEnumeratedSequential( + public static GeneratorColumnSchema makeEnumeratedSequential( String name, ValueType type, boolean isMetric, @@ -273,7 +336,7 @@ public static BenchmarkColumnSchema makeEnumeratedSequential( List enumeratedValues ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -285,7 +348,7 @@ public static BenchmarkColumnSchema makeEnumeratedSequential( return schema; } - public static BenchmarkColumnSchema makeDiscreteUniform( + public static GeneratorColumnSchema makeDiscreteUniform( String name, ValueType type, boolean isMetric, @@ -295,7 +358,7 @@ public static BenchmarkColumnSchema makeDiscreteUniform( int endInt ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -308,7 +371,7 @@ public static BenchmarkColumnSchema makeDiscreteUniform( return schema; } - public static BenchmarkColumnSchema makeEnumeratedDiscreteUniform( + public static GeneratorColumnSchema makeEnumeratedDiscreteUniform( String name, ValueType type, boolean isMetric, @@ -317,7 +380,7 @@ public static BenchmarkColumnSchema makeEnumeratedDiscreteUniform( List enumeratedValues ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -329,7 +392,31 @@ public static BenchmarkColumnSchema makeEnumeratedDiscreteUniform( return schema; } - public static BenchmarkColumnSchema makeContinuousUniform( + public static GeneratorColumnSchema makeLazyDiscreteUniform( + String name, + ValueType type, + boolean isMetric, + int rowSize, + Double nullProbability, + int startInt, + int endInt + ) + { + GeneratorColumnSchema schema = new GeneratorColumnSchema( + name, + type, + isMetric, + rowSize, + nullProbability, + ValueDistribution.LAZY_DISCRETE_UNIFORM + ); + schema.startInt = startInt; + schema.endInt = endInt; + return schema; + } + + + public static GeneratorColumnSchema makeContinuousUniform( String name, ValueType type, boolean isMetric, @@ -339,7 +426,7 @@ public static BenchmarkColumnSchema makeContinuousUniform( double endDouble ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -352,7 +439,7 @@ public static BenchmarkColumnSchema makeContinuousUniform( return schema; } - public static BenchmarkColumnSchema makeNormal( + public static GeneratorColumnSchema makeNormal( String name, ValueType type, boolean isMetric, @@ -363,7 +450,7 @@ public static BenchmarkColumnSchema makeNormal( boolean useRounding ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -376,7 +463,7 @@ public static BenchmarkColumnSchema makeNormal( return schema; } - public static BenchmarkColumnSchema makeZipf( + public static GeneratorColumnSchema makeZipf( String name, ValueType type, boolean isMetric, @@ -387,7 +474,7 @@ public static BenchmarkColumnSchema makeZipf( Double zipfExponent ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -401,7 +488,32 @@ public static BenchmarkColumnSchema makeZipf( return schema; } - public static BenchmarkColumnSchema makeEnumeratedZipf( + public static GeneratorColumnSchema makeLazyZipf( + String name, + ValueType type, + boolean isMetric, + int rowSize, + Double nullProbability, + int startInt, + int endInt, + Double zipfExponent + ) + { + GeneratorColumnSchema schema = new GeneratorColumnSchema( + name, + type, + isMetric, + rowSize, + nullProbability, + ValueDistribution.LAZY_ZIPF + ); + schema.startInt = startInt; + schema.endInt = endInt; + schema.zipfExponent = zipfExponent; + return schema; + } + + public static GeneratorColumnSchema makeEnumeratedZipf( String name, ValueType type, boolean isMetric, @@ -411,7 +523,7 @@ public static BenchmarkColumnSchema makeEnumeratedZipf( Double zipfExponent ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -424,7 +536,7 @@ public static BenchmarkColumnSchema makeEnumeratedZipf( return schema; } - public static BenchmarkColumnSchema makeEnumerated( + public static GeneratorColumnSchema makeEnumerated( String name, ValueType type, boolean isMetric, @@ -434,7 +546,7 @@ public static BenchmarkColumnSchema makeEnumerated( List enumeratedProbabilities ) { - BenchmarkColumnSchema schema = new BenchmarkColumnSchema( + GeneratorColumnSchema schema = new GeneratorColumnSchema( name, type, isMetric, @@ -468,4 +580,53 @@ public String toString() ", standardDeviation=" + standardDeviation + '}'; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + GeneratorColumnSchema that = (GeneratorColumnSchema) o; + return isMetric == that.isMetric && + rowSize == that.rowSize && + distributionType == that.distributionType && + name.equals(that.name) && + type == that.type && + Objects.equals(nullProbability, that.nullProbability) && + Objects.equals(enumeratedValues, that.enumeratedValues) && + Objects.equals(enumeratedProbabilities, that.enumeratedProbabilities) && + Objects.equals(startInt, that.startInt) && + Objects.equals(endInt, that.endInt) && + Objects.equals(startDouble, that.startDouble) && + Objects.equals(endDouble, that.endDouble) && + Objects.equals(zipfExponent, that.zipfExponent) && + Objects.equals(mean, that.mean) && + Objects.equals(standardDeviation, that.standardDeviation); + } + + @Override + public int hashCode() + { + return Objects.hash( + distributionType, + name, + type, + isMetric, + rowSize, + nullProbability, + enumeratedValues, + enumeratedProbabilities, + startInt, + endInt, + startDouble, + endDouble, + zipfExponent, + mean, + standardDeviation + ); + } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkSchemaInfo.java b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorSchemaInfo.java similarity index 86% rename from benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkSchemaInfo.java rename to processing/src/main/java/org/apache/druid/segment/generator/GeneratorSchemaInfo.java index f8c1b60dd373..cb1591ab897e 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/BenchmarkSchemaInfo.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/GeneratorSchemaInfo.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.benchmark.datagen; +package org.apache.druid.segment.generator; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -27,15 +27,15 @@ import java.util.List; import java.util.stream.Collectors; -public class BenchmarkSchemaInfo +public class GeneratorSchemaInfo { - private List columnSchemas; + private List columnSchemas; private List aggs; private Interval dataInterval; private boolean withRollup; - public BenchmarkSchemaInfo( - List columnSchemas, + public GeneratorSchemaInfo( + List columnSchemas, List aggs, Interval dataInterval, boolean withRollup @@ -47,7 +47,7 @@ public BenchmarkSchemaInfo( this.withRollup = withRollup; } - public List getColumnSchemas() + public List getColumnSchemas() { return columnSchemas; } @@ -56,7 +56,7 @@ public DimensionsSpec getDimensionsSpec() { List specs = getColumnSchemas().stream() .filter(x -> !x.isMetric()) - .map(BenchmarkColumnSchema::getDimensionSchema) + .map(GeneratorColumnSchema::getDimensionSchema) .collect(Collectors.toList()); return new DimensionsSpec(specs); diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/RealRoundingDistribution.java b/processing/src/main/java/org/apache/druid/segment/generator/RealRoundingDistribution.java similarity index 97% rename from benchmarks/src/test/java/org/apache/druid/benchmark/datagen/RealRoundingDistribution.java rename to processing/src/main/java/org/apache/druid/segment/generator/RealRoundingDistribution.java index 320a0cb9a352..6d663cfec484 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/RealRoundingDistribution.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/RealRoundingDistribution.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.benchmark.datagen; +package org.apache.druid.segment.generator; import org.apache.commons.math3.distribution.AbstractIntegerDistribution; import org.apache.commons.math3.distribution.AbstractRealDistribution; diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/SequentialDistribution.java b/processing/src/main/java/org/apache/druid/segment/generator/SequentialDistribution.java similarity index 97% rename from benchmarks/src/test/java/org/apache/druid/benchmark/datagen/SequentialDistribution.java rename to processing/src/main/java/org/apache/druid/segment/generator/SequentialDistribution.java index c10d39255355..514540be0631 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/datagen/SequentialDistribution.java +++ b/processing/src/main/java/org/apache/druid/segment/generator/SequentialDistribution.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.benchmark.datagen; +package org.apache.druid.segment.generator; import org.apache.commons.math3.distribution.EnumeratedDistribution; import org.apache.commons.math3.util.Pair; 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 c169d15f2230..649aea9b6997 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 @@ -27,6 +27,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.primitives.Ints; import com.google.common.primitives.Longs; import com.google.errorprone.annotations.concurrent.GuardedBy; @@ -89,6 +90,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.ConcurrentMap; @@ -327,9 +329,10 @@ protected IncrementalIndex( } //__time capabilities - ColumnCapabilitiesImpl timeCapabilities = new ColumnCapabilitiesImpl().setIsComplete(true); - timeCapabilities.setType(ValueType.LONG); - columnCapabilities.put(ColumnHolder.TIME_COLUMN_NAME, timeCapabilities); + columnCapabilities.put( + ColumnHolder.TIME_COLUMN_NAME, + ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG) + ); // This should really be more generic List spatialDimensions = dimensionsSpec.getSpatialDimensions(); @@ -640,12 +643,15 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) } final List rowDimensions = row.getDimensions(); - Object[] dims; List overflow = null; long dimsKeySize = 0; List parseExceptionMessages = new ArrayList<>(); synchronized (dimensionDescs) { + // all known dimensions are assumed missing until we encounter in the rowDimensions + Set absentDimensions = Sets.newHashSet(dimensionDescs.keySet()); + + // first, process dimension values present in the row dims = new Object[dimensionDescs.size()]; for (String dimension : rowDimensions) { if (Strings.isNullOrEmpty(dimension)) { @@ -656,18 +662,13 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) DimensionDesc desc = dimensionDescs.get(dimension); if (desc != null) { capabilities = desc.getCapabilities(); + absentDimensions.remove(dimension); } else { wasNewDim = true; capabilities = columnCapabilities.get(dimension); if (capabilities == null) { - capabilities = new ColumnCapabilitiesImpl(); // For schemaless type discovery, assume everything is a String for now, can change later. - capabilities.setType(ValueType.STRING); - capabilities.setDictionaryEncoded(true); - capabilities.setHasBitmapIndexes(true); - capabilities.setDictionaryValuesSorted(false); - capabilities.setDictionaryValuesUnique(true); - capabilities.setIsComplete(true); + capabilities = makeCapabilitiesFromValueType(ValueType.STRING); columnCapabilities.put(dimension, capabilities); } DimensionHandler handler = DimensionHandlerUtils.getHandlerFromCapabilities(dimension, capabilities, null); @@ -677,23 +678,24 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) DimensionIndexer indexer = desc.getIndexer(); Object dimsKey = null; try { - dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent( - row.getRaw(dimension), - true - ); + dimsKey = indexer.processRowValsToUnsortedEncodedKeyComponent(row.getRaw(dimension), true); } catch (ParseException pe) { parseExceptionMessages.add(pe.getMessage()); } dimsKeySize += indexer.estimateEncodedKeyComponentSize(dimsKey); // Set column capabilities as data is coming in - if (!capabilities.hasMultipleValues() && + if (!capabilities.hasMultipleValues().isTrue() && dimsKey != null && handler.getLengthOfEncodedKeyComponent(dimsKey) > 1) { capabilities.setHasMultipleValues(true); } if (wasNewDim) { + // unless this is the first row we are processing, all newly discovered columns will be sparse + if (maxIngestedEventTime != null) { + indexer.setSparseIndexed(); + } if (overflow == null) { overflow = new ArrayList<>(); } @@ -713,6 +715,11 @@ IncrementalIndexRowResult toIncrementalIndexRow(InputRow row) dims[desc.getIndex()] = dimsKey; } } + + // process any dimensions with missing values in the row + for (String missing : absentDimensions) { + dimensionDescs.get(missing).getIndexer().setSparseIndexed(); + } } if (overflow != null) { @@ -923,16 +930,16 @@ public List getDimensionOrder() private ColumnCapabilitiesImpl makeCapabilitiesFromValueType(ValueType type) { - ColumnCapabilitiesImpl capabilities = new ColumnCapabilitiesImpl(); - capabilities.setDictionaryEncoded(type == ValueType.STRING); - capabilities.setHasBitmapIndexes(type == ValueType.STRING); if (type == ValueType.STRING) { - capabilities.setDictionaryValuesUnique(true); - capabilities.setDictionaryValuesSorted(false); + // we start out as not having multiple values, but this might change as we encounter them + return new ColumnCapabilitiesImpl().setType(type) + .setHasBitmapIndexes(true) + .setDictionaryEncoded(true) + .setDictionaryValuesUnique(true) + .setDictionaryValuesSorted(false); + } else { + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(type); } - capabilities.setType(type); - capabilities.setIsComplete(true); - return capabilities; } /** @@ -988,6 +995,7 @@ public StorageAdapter toStorageAdapter() return new IncrementalIndexStorageAdapter(this); } + @Nullable public ColumnCapabilities getCapabilities(String column) { return columnCapabilities.get(column); @@ -1124,18 +1132,18 @@ public MetricDesc(int index, AggregatorFactory factory) this.name = factory.getName(); String typeInfo = factory.getTypeName(); - this.capabilities = new ColumnCapabilitiesImpl().setIsComplete(true); if ("float".equalsIgnoreCase(typeInfo)) { - capabilities.setType(ValueType.FLOAT); + capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.FLOAT); this.type = typeInfo; } else if ("long".equalsIgnoreCase(typeInfo)) { - capabilities.setType(ValueType.LONG); + capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); this.type = typeInfo; } else if ("double".equalsIgnoreCase(typeInfo)) { - capabilities.setType(ValueType.DOUBLE); + capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.DOUBLE); this.type = typeInfo; } else { - capabilities.setType(ValueType.COMPLEX); + // in an ideal world complex type reports its actual column capabilities... + capabilities = ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.COMPLEX); this.type = ComplexMetrics.getSerdeForType(typeInfo).getTypeName(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index cc792a5314c6..8e8520d458b7 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -39,7 +39,6 @@ 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.ValueType; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; import org.apache.druid.segment.filter.BooleanValueMatcher; @@ -150,16 +149,23 @@ public ColumnCapabilities getColumnCapabilities(String column) // 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 // they appear multi-valued. + // + // Note that this could be improved if we snapshot the capabilities at cursor creation time and feed those through + // to the StringDimensionIndexer so the selector built on top of it can produce values from the snapshot state of + // multi-valuedness at cursor creation time, instead of the latest state, and getSnapshotColumnCapabilities could + // be removed. + return ColumnCapabilitiesImpl.snapshot(index.getCapabilities(column), true); + } - final ColumnCapabilities capabilitiesFromIndex = index.getCapabilities(column); - final IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(column); - if (dimensionDesc != null && dimensionDesc.getCapabilities().getType() == ValueType.STRING) { - final ColumnCapabilitiesImpl retVal = ColumnCapabilitiesImpl.copyOf(capabilitiesFromIndex); - retVal.setHasMultipleValues(true); - return retVal; - } else { - return capabilitiesFromIndex; - } + /** + * 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 + * 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.getCapabilities(column)); } @Override 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 d321af958008..500e030c8c7d 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 @@ -20,44 +20,48 @@ package org.apache.druid.segment.join; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.segment.AbstractSegment; +import org.apache.druid.java.util.common.guava.CloseQuietly; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; import java.util.List; +import java.util.Optional; /** * Represents a deep, left-heavy join of a left-hand side baseSegment onto a series of right-hand side clauses. * * In other words, logically the operation is: join(join(join(baseSegment, clauses[0]), clauses[1]), clauses[2]) etc. */ -public class HashJoinSegment extends AbstractSegment +public class HashJoinSegment implements SegmentReference { - private final Segment baseSegment; + private final SegmentReference baseSegment; private final List clauses; - private final JoinFilterPreAnalysis joinFilterPreAnalysis; + private final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup; /** * @param baseSegment 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 computed by {@link org.apache.druid.segment.join.filter.JoinFilterAnalyzer#computeJoinFilterPreAnalysis} + * @param joinFilterPreAnalysisGroup Pre-analysis group that holds all of the JoinFilterPreAnalysis results within + * the scope of a query */ public HashJoinSegment( - Segment baseSegment, + SegmentReference baseSegment, List clauses, - JoinFilterPreAnalysis joinFilterPreAnalysis + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup ) { this.baseSegment = baseSegment; this.clauses = clauses; - this.joinFilterPreAnalysis = joinFilterPreAnalysis; + this.joinFilterPreAnalysisGroup = joinFilterPreAnalysisGroup; // Verify 'clauses' is nonempty (otherwise it's a waste to create this object, and the caller should know) if (clauses.isEmpty()) { @@ -90,7 +94,7 @@ public QueryableIndex asQueryableIndex() @Override public StorageAdapter asStorageAdapter() { - return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, joinFilterPreAnalysis); + return new HashJoinSegmentStorageAdapter(baseSegment.asStorageAdapter(), clauses, joinFilterPreAnalysisGroup); } @Override @@ -98,4 +102,36 @@ public void close() throws IOException { baseSegment.close(); } + + @Override + public Optional acquireReferences() + { + Closer closer = Closer.create(); + try { + boolean acquireFailed = baseSegment.acquireReferences().map(closeable -> { + closer.register(closeable); + return false; + }).orElse(true); + + for (JoinableClause joinClause : clauses) { + if (acquireFailed) { + break; + } + acquireFailed |= joinClause.acquireReferences().map(closeable -> { + closer.register(closeable); + return false; + }).orElse(true); + } + if (acquireFailed) { + CloseQuietly.close(closer); + return Optional.empty(); + } else { + return Optional.of(closer); + } + } + catch (Exception ex) { + CloseQuietly.close(closer); + return Optional.empty(); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java index 65c25d498b56..8e2518e5cc78 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapter.java @@ -21,7 +21,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -38,6 +37,7 @@ import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.filter.JoinFilterSplit; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -47,7 +47,6 @@ import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -55,22 +54,23 @@ public class HashJoinSegmentStorageAdapter implements StorageAdapter { private final StorageAdapter baseAdapter; private final List clauses; - private final JoinFilterPreAnalysis joinFilterPreAnalysis; + private final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup; /** * @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 joinFilterPreAnalysisGroup Pre-analysis group that holds all of the JoinFilterPreAnalysis results within + * the scope of a query */ HashJoinSegmentStorageAdapter( StorageAdapter baseAdapter, List clauses, - final JoinFilterPreAnalysis joinFilterPreAnalysis + final JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup ) { this.baseAdapter = baseAdapter; this.clauses = clauses; - this.joinFilterPreAnalysis = joinFilterPreAnalysis; + this.joinFilterPreAnalysisGroup = joinFilterPreAnalysisGroup; } @Override @@ -209,13 +209,16 @@ public Sequence makeCursors( @Nullable final QueryMetrics queryMetrics ) { - if (!Objects.equals(joinFilterPreAnalysis.getOriginalFilter(), filter)) { - throw new ISE( - "Filter provided to cursor [%s] does not match join pre-analysis filter [%s]", + JoinFilterPreAnalysis jfpa; + if (joinFilterPreAnalysisGroup.isSingleLevelMode()) { + jfpa = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + } else { + jfpa = joinFilterPreAnalysisGroup.getAnalysis( filter, - joinFilterPreAnalysis.getOriginalFilter() + virtualColumns ); } + final List preJoinVirtualColumns = new ArrayList<>(); final List postJoinVirtualColumns = new ArrayList<>(); @@ -225,7 +228,7 @@ public Sequence makeCursors( postJoinVirtualColumns ); - JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + JoinFilterSplit joinFilterSplit = JoinFilterAnalyzer.splitFilter(jfpa); preJoinVirtualColumns.addAll(joinFilterSplit.getPushDownVirtualColumns()); // Soon, we will need a way to push filters past a join when possible. This could potentially be done right here diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java index 1ededff5e8b8..7ad7799a1099 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/Joinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinable.java @@ -20,10 +20,12 @@ package org.apache.druid.segment.join; import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.ReferenceCountedObject; import org.apache.druid.segment.column.ColumnCapabilities; import javax.annotation.Nullable; import java.util.List; +import java.util.Optional; import java.util.Set; /** @@ -32,7 +34,7 @@ * This class's most important method is {@link #makeJoinMatcher}. Its main user is * {@link HashJoinEngine#makeJoinCursor}. */ -public interface Joinable +public interface Joinable extends ReferenceCountedObject { int CARDINALITY_UNKNOWN = -1; @@ -88,9 +90,11 @@ JoinMatcher makeJoinMatcher( * returned than this limit, return an empty set. * @param allowNonKeyColumnSearch If true, allow searchs on non-key columns. If this is false, * a search on a non-key column should return an empty set. - * @return The set of correlated column values. If we cannot determine correlated values, return an empty set. + * @return The set of correlated column values. If we cannot determine correlated values, return absent. + * + * In case either the search or retrieval column names are not found, this will return absent. */ - Set getCorrelatedColumnValues( + Optional> getCorrelatedColumnValues( String searchColumnName, String searchColumnValue, String retrievalColumnName, diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java index a2ddefe1bf0a..2f8bd3ac322f 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableClause.java @@ -21,9 +21,12 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.segment.ReferenceCountedObject; +import java.io.Closeable; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.stream.Collectors; /** @@ -33,7 +36,7 @@ * * Created from {@link org.apache.druid.query.planning.PreJoinableClause} by {@link Joinables#createSegmentMapFn}. */ -public class JoinableClause +public class JoinableClause implements ReferenceCountedObject { private final String prefix; private final Joinable joinable; @@ -151,4 +154,10 @@ public String toString() ", condition=" + condition + '}'; } + + @Override + public Optional acquireReferences() + { + return joinable.acquireReferences(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactory.java index fc63f1cfbee3..723aba57faa9 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/JoinableFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/join/JoinableFactory.java @@ -30,6 +30,13 @@ */ public interface JoinableFactory { + /** + * Returns true if a {@link Joinable} **may** be created for a given {@link DataSource}, but is not a guarantee that + * {@link #build} will return a non-empty result. Successfully building a {@link Joinable} might require specific + * criteria of the {@link JoinConditionAnalysis}. + */ + boolean isDirectlyJoinable(DataSource dataSource); + /** * Create a Joinable object. This may be an expensive operation involving loading data, creating a hash table, etc. * diff --git a/processing/src/main/java/org/apache/druid/segment/join/Joinables.java b/processing/src/main/java/org/apache/druid/segment/join/Joinables.java index 261f2d6c8385..6ebe970ae29e 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/Joinables.java +++ b/processing/src/main/java/org/apache/druid/segment/join/Joinables.java @@ -20,24 +20,24 @@ package org.apache.druid.segment.join; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.JoinDataSource; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.planning.PreJoinableClause; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.ColumnHolder; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Comparator; import java.util.List; -import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; -import java.util.stream.Collectors; /** * Utility methods for working with {@link Joinable} related classes. @@ -74,33 +74,19 @@ public static boolean isPrefixedBy(final String columnName, final String prefix) /** * Creates a Function that maps base segments to {@link HashJoinSegment} if needed (i.e. if the number of join * clauses is > 0). If mapping is not needed, this method will return {@link Function#identity()}. - * - * @param clauses pre-joinable clauses - * @param joinableFactory factory for joinables - * @param cpuTimeAccumulator an accumulator that we will add CPU nanos to; this is part of the function to encourage - * callers to remember to track metrics on CPU time required for creation of Joinables - * @param enableFilterPushDown whether to enable filter push down optimizations to the base segment. In production - * this should generally be {@code QueryContexts.getEnableJoinFilterPushDown(query)}. - * @param enableFilterRewrite whether to enable filter rewrite optimizations for RHS columns. In production - * this should generally be {@code QueryContexts.getEnableJoinFilterRewrite(query)}. - * @param enableRewriteValueColumnFilters whether to enable filter rewrite optimizations for RHS columns that are not - * key columns. In production this should generally - * be {@code QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query)}. - * @param filterRewriteMaxSize the max allowed size of correlated value sets for RHS rewrites. In production - * this should generally be {@code QueryContexts.getJoinFilterRewriteMaxSize(query)}. - * @param originalFilter The original filter from the query. - * @param virtualColumns The virtual columns from the query. + * @param clauses Pre-joinable clauses + * @param joinableFactory Factory for joinables + * @param cpuTimeAccumulator An accumulator that we will add CPU nanos to; this is part of the function to encourage + * callers to remember to track metrics on CPU time required for creation of Joinables + * @param joinFilterRewriteConfig Configuration options for the join filter rewrites + * @param query The query being processed */ - public static Function createSegmentMapFn( + public static Function createSegmentMapFn( final List clauses, final JoinableFactory joinableFactory, final AtomicLong cpuTimeAccumulator, - final boolean enableFilterPushDown, - final boolean enableFilterRewrite, - final boolean enableRewriteValueColumnFilters, - final long filterRewriteMaxSize, - final Filter originalFilter, - final VirtualColumns virtualColumns + final JoinFilterRewriteConfig joinFilterRewriteConfig, + final Query query ) { // compute column correlations here and RHS correlated values @@ -110,59 +96,84 @@ public static Function createSegmentMapFn( if (clauses.isEmpty()) { return Function.identity(); } else { - final List joinableClauses = createJoinableClauses(clauses, joinableFactory); - JoinFilterPreAnalysis jfpa = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, - originalFilter, - enableFilterPushDown, - enableFilterRewrite, - enableRewriteValueColumnFilters, - filterRewriteMaxSize + final JoinableClauses joinableClauses = JoinableClauses.createClauses(clauses, joinableFactory); + + List joinQueryLevels = new ArrayList<>(); + Joinables.gatherAllJoinQueryLevels(query, joinQueryLevels); + + final JoinFilterPreAnalysisGroup preAnalysisGroup = new JoinFilterPreAnalysisGroup( + joinFilterRewriteConfig, + joinQueryLevels.size() <= 1 // use single-level mode if there's one or fewer query levels with joins ); - return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses, jfpa); + + for (Query joinQuery : joinQueryLevels) { + preAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( + joinQuery.getFilter() == null ? null : joinQuery.getFilter().toFilter(), + joinableClauses.getJoinableClauses(), + joinQuery.getVirtualColumns() + ); + } + + return baseSegment -> new HashJoinSegment(baseSegment, joinableClauses.getJoinableClauses(), preAnalysisGroup); } } ); } /** - * Returns a list of {@link JoinableClause} corresponding to a list of {@link PreJoinableClause}. This will call - * {@link JoinableFactory#build} on each one and therefore may be an expensive operation. + * Walks a query and its subqueries, finding any queries that read from a JoinDatasource, + * and adding them to a list provided by the caller. + * + * @param currentLevelQuery The query to analyze + * @param allJoinQueryLevels A mutable list provided by the caller. */ - private static List createJoinableClauses( - final List clauses, - final JoinableFactory joinableFactory - ) + public static void gatherAllJoinQueryLevels(Query currentLevelQuery, List allJoinQueryLevels) { - // Since building a JoinableClause can be expensive, check for prefix conflicts before building - checkPreJoinableClausesForDuplicatesAndShadowing(clauses); - - return clauses.stream().map(preJoinableClause -> { - final Optional joinable = joinableFactory.build( - preJoinableClause.getDataSource(), - preJoinableClause.getCondition() + DataSource currentDatasource = currentLevelQuery.getDataSource(); + if (currentDatasource instanceof QueryDataSource) { + gatherAllJoinQueryLevels( + ((QueryDataSource) currentDatasource).getQuery(), + allJoinQueryLevels ); - - return new JoinableClause( - preJoinableClause.getPrefix(), - joinable.orElseThrow(() -> new ISE("dataSource is not joinable: %s", preJoinableClause.getDataSource())), - preJoinableClause.getJoinType(), - preJoinableClause.getCondition() + } + if (currentDatasource instanceof JoinDataSource) { + allJoinQueryLevels.add(currentLevelQuery); + gatherAllJoinQueryLevelsJoinDatasourceHelper( + (JoinDataSource) currentDatasource, + allJoinQueryLevels ); - }).collect(Collectors.toList()); + } } - private static void checkPreJoinableClausesForDuplicatesAndShadowing( - final List preJoinableClauses + private static void gatherAllJoinQueryLevelsJoinDatasourceHelper( + JoinDataSource joinDatasource, + List allJoinQueryLevels ) { - List prefixes = new ArrayList<>(); - for (PreJoinableClause clause : preJoinableClauses) { - prefixes.add(clause.getPrefix()); + if (joinDatasource.getLeft() instanceof QueryDataSource) { + gatherAllJoinQueryLevels( + ((QueryDataSource) joinDatasource.getLeft()).getQuery(), + allJoinQueryLevels + ); + } + if (joinDatasource.getLeft() instanceof JoinDataSource) { + gatherAllJoinQueryLevelsJoinDatasourceHelper( + (JoinDataSource) joinDatasource.getLeft(), + allJoinQueryLevels + ); + } + if (joinDatasource.getRight() instanceof QueryDataSource) { + gatherAllJoinQueryLevels( + ((QueryDataSource) joinDatasource.getRight()).getQuery(), + allJoinQueryLevels + ); + } + if (joinDatasource.getRight() instanceof JoinDataSource) { + gatherAllJoinQueryLevelsJoinDatasourceHelper( + (JoinDataSource) joinDatasource.getRight(), + allJoinQueryLevels + ); } - - checkPrefixesForDuplicatesAndShadowing(prefixes); } /** diff --git a/processing/src/main/java/org/apache/druid/segment/join/MapJoinableFactory.java b/processing/src/main/java/org/apache/druid/segment/join/MapJoinableFactory.java index beb8106225d4..abf4b6ae4d06 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/MapJoinableFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/join/MapJoinableFactory.java @@ -43,6 +43,17 @@ public MapJoinableFactory(Map, JoinableFactory> join this.joinableFactories = new IdentityHashMap<>(joinableFactories); } + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + JoinableFactory factory = joinableFactories.get(dataSource.getClass()); + if (factory == null) { + return false; + } else { + return factory.isDirectlyJoinable(dataSource); + } + } + @Override public Optional build(DataSource dataSource, JoinConditionAnalysis condition) { diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/Equiconditions.java b/processing/src/main/java/org/apache/druid/segment/join/filter/Equiconditions.java new file mode 100644 index 000000000000..33a07987b8b9 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/Equiconditions.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.segment.join.filter; + +import org.apache.druid.math.expr.Expr; +import org.apache.druid.query.filter.Filter; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Set; + +public class Equiconditions +{ + @Nonnull private final Map> equiconditions; + + public Equiconditions(Map> equiconditions) + { + this.equiconditions = equiconditions; + } + + /** + * @param filterClause the filter. + * @return true if direct join filter rewrite is supported for the provided filter + */ + public boolean doesFilterSupportDirectJoinFilterRewrite(Filter filterClause) + { + if (filterClause.supportsRequiredColumnRewrite()) { + Set requiredColumns = filterClause.getRequiredColumns(); + if (requiredColumns.size() == 1) { + String reqColumn = requiredColumns.iterator().next(); + return equiconditions.containsKey(reqColumn); + } + } + return false; + } + + @Nullable + public Set getLhsExprs(String rhsColumn) + { + return equiconditions.get(rhsColumn); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java index 77f4c9d987f1..06475ad89e51 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterAnalyzer.java @@ -25,28 +25,20 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.InDimFilter; -import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.filter.AndFilter; +import org.apache.druid.segment.filter.FalseFilter; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.OrFilter; import org.apache.druid.segment.filter.SelectorFilter; -import org.apache.druid.segment.join.Equality; -import org.apache.druid.segment.join.JoinConditionAnalysis; -import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -76,13 +68,12 @@ * * The result of this pre-analysis method should be passed into the next step of join filter analysis, described below. * - * The {@link #splitFilter(JoinFilterPreAnalysis)} method takes the pre-analysis result and optionally applies the\ + * The {@link #splitFilter(JoinFilterPreAnalysis)} method takes the pre-analysis result and optionally applies the * filter rewrite and push down operations on a per-segment level. */ public class JoinFilterAnalyzer { private static final String PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE = "JOIN-FILTER-PUSHDOWN-VIRTUAL-COLUMN-"; - private static final ColumnSelectorFactory ALL_NULL_COLUMN_SELECTOR_FACTORY = new AllNullColumnSelectorFactory(); /** * Before making per-segment filter splitting decisions, we first do a pre-analysis step @@ -95,64 +86,34 @@ public class JoinFilterAnalyzer * @param joinableClauses The joinable clauses from the query * @param virtualColumns The virtual columns from the query * @param originalFilter The original filter from the query - * @param enableFilterPushDown Whether to enable filter push down - * @param enableFilterRewrite Whether to enable rewrites of filters involving RHS columns - * @param enableRewriteValueColumnFilters Whether to enable rewrites of filters invovling RHS non-key columns - * @param filterRewriteMaxSize The maximum size of the correlated value set for rewritten filters. - * If the correlated value set size exceeds this, the filter will not be - * rewritten and pushed down. - * + * @param joinFilterRewriteConfig Configuration options for the join rewrites * @return A JoinFilterPreAnalysis containing information determined in this pre-analysis step. */ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( - List joinableClauses, + JoinableClauses joinableClauses, VirtualColumns virtualColumns, Filter originalFilter, - boolean enableFilterPushDown, - boolean enableFilterRewrite, - boolean enableRewriteValueColumnFilters, - long filterRewriteMaxSize + JoinFilterRewriteConfig joinFilterRewriteConfig ) { final List preJoinVirtualColumns = new ArrayList<>(); final List postJoinVirtualColumns = new ArrayList<>(); - splitVirtualColumns(joinableClauses, virtualColumns, preJoinVirtualColumns, postJoinVirtualColumns); - - if (originalFilter == null || !enableFilterPushDown) { - return new JoinFilterPreAnalysis( - joinableClauses, - originalFilter, - postJoinVirtualColumns, - null, - null, - null, - null, - enableFilterPushDown, - enableFilterRewrite, - Collections.emptyMap() - ); + joinableClauses.splitVirtualColumns(virtualColumns, preJoinVirtualColumns, postJoinVirtualColumns); + JoinFilterPreAnalysis.Builder preAnalysisBuilder = + new JoinFilterPreAnalysis.Builder(joinFilterRewriteConfig, joinableClauses, originalFilter, postJoinVirtualColumns); + if (originalFilter == null || !joinFilterRewriteConfig.isEnableFilterPushDown()) { + return preAnalysisBuilder.build(); } - Filter normalizedFilter = Filters.toCnf(originalFilter); - - // List of candidates for pushdown - // CNF normalization will generate either - // - an AND filter with multiple subfilters - // - or a single non-AND subfilter which cannot be split further - Set normalizedOrClauses; - if (normalizedFilter instanceof AndFilter) { - normalizedOrClauses = ((AndFilter) normalizedFilter).getFilters(); - } else { - normalizedOrClauses = Collections.singleton(normalizedFilter); - } + Set normalizedOrClauses = Filters.toNormalizedOrClauses(originalFilter); List normalizedBaseTableClauses = new ArrayList<>(); List normalizedJoinTableClauses = new ArrayList<>(); for (Filter orClause : normalizedOrClauses) { Set reqColumns = orClause.getRequiredColumns(); - if (areSomeColumnsFromJoin(joinableClauses, reqColumns) || areSomeColumnsFromPostJoinVirtualColumns( + if (joinableClauses.areSomeColumnsFromJoin(reqColumns) || areSomeColumnsFromPostJoinVirtualColumns( postJoinVirtualColumns, reqColumns )) { @@ -161,264 +122,26 @@ public static JoinFilterPreAnalysis computeJoinFilterPreAnalysis( normalizedBaseTableClauses.add(orClause); } } - - if (!enableFilterRewrite) { - return new JoinFilterPreAnalysis( - joinableClauses, - originalFilter, - postJoinVirtualColumns, - normalizedBaseTableClauses, - normalizedJoinTableClauses, - null, - null, - enableFilterPushDown, - enableFilterRewrite, - Collections.emptyMap() - ); + preAnalysisBuilder + .withNormalizedBaseTableClauses(normalizedBaseTableClauses) + .withNormalizedJoinTableClauses(normalizedJoinTableClauses); + if (!joinFilterRewriteConfig.isEnableFilterRewrite()) { + return preAnalysisBuilder.build(); } // build the equicondition map, used for determining how the tables are connected through joins - Map> equiconditions = new HashMap<>(); - for (JoinableClause clause : joinableClauses) { - for (Equality equality : clause.getCondition().getEquiConditions()) { - Set exprsForRhs = equiconditions.computeIfAbsent( - clause.getPrefix() + equality.getRightColumn(), - (rhs) -> new HashSet<>() - ); - exprsForRhs.add(equality.getLeftExpr()); - } - } + Equiconditions equiconditions = preAnalysisBuilder.computeEquiconditionsFromJoinableClauses(); - // Determine candidates for filter rewrites. - // A candidate is an RHS column that appears in a filter, along with the value being filtered on, plus - // the joinable clause associated with the table that the RHS column is from. - Set rhsRewriteCandidates = new LinkedHashSet<>(); - for (Filter orClause : normalizedJoinTableClauses) { - if (filterMatchesNull(orClause)) { - continue; - } - - if (orClause instanceof OrFilter) { - for (Filter subFilter : ((OrFilter) orClause).getFilters()) { - Optional rhsRewriteCandidate = determineRhsRewriteCandidatesForSingleFilter( - subFilter, - equiconditions, - joinableClauses - ); - - if (rhsRewriteCandidate.isPresent()) { - rhsRewriteCandidates.add(rhsRewriteCandidate.get()); - } - } - continue; - } - - Optional rhsRewriteCandidate = determineRhsRewriteCandidatesForSingleFilter( - orClause, - equiconditions, - joinableClauses - ); - - if (rhsRewriteCandidate.isPresent()) { - rhsRewriteCandidates.add(rhsRewriteCandidate.get()); - } - } - - // Build a map of RHS table prefix -> JoinFilterColumnCorrelationAnalysis based on the RHS rewrite candidates - Map>> correlationsByPrefix = new HashMap<>(); - Map> directRewriteCorrelations = new HashMap<>(); - - for (RhsRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { - if (rhsRewriteCandidate.isDirectRewrite()) { - directRewriteCorrelations.computeIfAbsent( - rhsRewriteCandidate.getRhsColumn(), - c -> { - Optional> correlatedBaseTableColumns = - findCorrelatedBaseTableColumns( - joinableClauses, - c, - rhsRewriteCandidate, - equiconditions - ); - if (!correlatedBaseTableColumns.isPresent()) { - return Optional.empty(); - } else { - JoinFilterColumnCorrelationAnalysis baseColumnAnalysis = correlatedBaseTableColumns.get().get(c); - // for direct rewrites, there will only be one analysis keyed by the RHS column - assert (baseColumnAnalysis != null); - return Optional.of(correlatedBaseTableColumns.get().get(c)); - } - - } - ); - } else { - correlationsByPrefix.computeIfAbsent( - rhsRewriteCandidate.getJoinableClause().getPrefix(), - p -> findCorrelatedBaseTableColumns( - joinableClauses, - p, - rhsRewriteCandidate, - equiconditions - ) - ); - } - } - - // Using the RHS table prefix -> JoinFilterColumnCorrelationAnalysis created in the previous step, - // build a map of rhsFilterColumn -> Pair(rhsFilterColumn, rhsFilterValue) -> correlatedValues for specific filter pair - // The Pair(rhsFilterColumn, rhsFilterValue) -> correlatedValues mappings are stored in the - // JoinFilterColumnCorrelationAnalysis objects, which are shared across all rhsFilterColumn entries that belong - // to the same RHS table. - // - // The value is a List instead of a single value because a table can be joined - // to another via multiple columns. - // (See JoinFilterAnalyzerTest.test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnRHS for an example) - Map> correlationsByFilteringColumn = new LinkedHashMap<>(); - Map> correlationsByDirectFilteringColumn = new LinkedHashMap<>(); - for (RhsRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates) { - if (rhsRewriteCandidate.isDirectRewrite()) { - List perColumnCorrelations = - correlationsByDirectFilteringColumn.computeIfAbsent( - rhsRewriteCandidate.getRhsColumn(), - (rhsCol) -> { - return new ArrayList<>(); - } - ); - perColumnCorrelations.add( - directRewriteCorrelations.get(rhsRewriteCandidate.getRhsColumn()).get() - ); - continue; - } - - Optional> correlationsForPrefix = correlationsByPrefix.get( - rhsRewriteCandidate.getJoinableClause().getPrefix() - ); - if (correlationsForPrefix.isPresent()) { - for (Map.Entry correlationForPrefix : correlationsForPrefix.get() - .entrySet()) { - List perColumnCorrelations = - correlationsByFilteringColumn.computeIfAbsent( - rhsRewriteCandidate.getRhsColumn(), - (rhsCol) -> { - return new ArrayList<>(); - } - ); - perColumnCorrelations.add(correlationForPrefix.getValue()); - correlationForPrefix.getValue().getCorrelatedValuesMap().computeIfAbsent( - Pair.of(rhsRewriteCandidate.getRhsColumn(), rhsRewriteCandidate.getValueForRewrite()), - (rhsVal) -> { - Set correlatedValues = getCorrelatedValuesForPushDown( - rhsRewriteCandidate.getRhsColumn(), - rhsRewriteCandidate.getValueForRewrite(), - correlationForPrefix.getValue().getJoinColumn(), - rhsRewriteCandidate.getJoinableClause(), - enableRewriteValueColumnFilters, - filterRewriteMaxSize - ); - - if (correlatedValues.isEmpty()) { - return Optional.empty(); - } else { - return Optional.of(correlatedValues); - } - } - ); - } - } else { - correlationsByFilteringColumn.put(rhsRewriteCandidate.getRhsColumn(), null); - } - } - - // Go through each per-column analysis list and prune duplicates - for (Map.Entry> correlation : correlationsByFilteringColumn - .entrySet()) { - if (correlation.getValue() != null) { - List dedupList = eliminateCorrelationDuplicates( - correlation.getValue() - ); - correlationsByFilteringColumn.put(correlation.getKey(), dedupList); - } - } - for (Map.Entry> correlation : correlationsByDirectFilteringColumn - .entrySet()) { - if (correlation.getValue() != null) { - List dedupList = eliminateCorrelationDuplicates( - correlation.getValue() - ); - correlationsByDirectFilteringColumn.put(correlation.getKey(), dedupList); - } - } - - - return new JoinFilterPreAnalysis( - joinableClauses, - originalFilter, - postJoinVirtualColumns, - normalizedBaseTableClauses, + JoinFilterCorrelations correlations = JoinFilterCorrelations.computeJoinFilterCorrelations( normalizedJoinTableClauses, - correlationsByFilteringColumn, - correlationsByDirectFilteringColumn, - enableFilterPushDown, - enableFilterRewrite, - equiconditions + equiconditions, + joinableClauses, + joinFilterRewriteConfig.isEnableRewriteValueColumnFilters(), + joinFilterRewriteConfig.getFilterRewriteMaxSize() ); - } - - private static Optional determineRhsRewriteCandidatesForSingleFilter( - Filter orClause, - Map> equiconditions, - List joinableClauses - ) - { - // Check if the filter clause is on the RHS join column. If so, we can rewrite the clause to filter on the - // LHS join column instead. - // Currently, we only support rewrites of filters that operate on a single column for simplicity. - Set requiredColumns = orClause.getRequiredColumns(); - if (orClause.supportsRequiredColumnRewrite() && - doesRequiredColumnSetSupportDirectJoinFilterRewrite(requiredColumns, equiconditions)) { - String reqColumn = requiredColumns.iterator().next(); - JoinableClause joinableClause = isColumnFromJoin(joinableClauses, reqColumn); - - return Optional.of( - new RhsRewriteCandidate( - joinableClause, - reqColumn, - null, - true - ) - ); - } else if (orClause instanceof SelectorFilter) { - // this is a candidate for RHS filter rewrite, determine column correlations and correlated values - String reqColumn = ((SelectorFilter) orClause).getDimension(); - String reqValue = ((SelectorFilter) orClause).getValue(); - JoinableClause joinableClause = isColumnFromJoin(joinableClauses, reqColumn); - if (joinableClause != null) { - return Optional.of( - new RhsRewriteCandidate( - joinableClause, - reqColumn, - reqValue, - false - ) - ); - } - } - return Optional.empty(); - } - - private static boolean doesRequiredColumnSetSupportDirectJoinFilterRewrite( - Set requiredColumns, - Map> equiconditions - ) - { - if (requiredColumns.size() == 1) { - String reqColumn = requiredColumns.iterator().next(); - if (equiconditions.containsKey(reqColumn)) { - return true; - } - } - return false; + return preAnalysisBuilder.withCorrelations(correlations) + .build(); } /** @@ -444,7 +167,7 @@ public static JoinFilterSplit splitFilter( Map pushDownVirtualColumnsForLhsExprs = new HashMap<>(); for (Filter baseTableFilter : joinFilterPreAnalysis.getNormalizedBaseTableClauses()) { - if (!filterMatchesNull(baseTableFilter)) { + if (!Filters.filterMatchesNull(baseTableFilter)) { leftFilters.add(baseTableFilter); } else { rightFilters.add(baseTableFilter); @@ -500,7 +223,7 @@ private static JoinFilterAnalysis analyzeJoinFilterClause( // NULL matching conditions are not currently pushed down. // They require special consideration based on the join type, and for simplicity of the initial implementation // this is not currently handled. - if (!joinFilterPreAnalysis.isEnableFilterRewrite() || filterMatchesNull(filterClause)) { + if (!joinFilterPreAnalysis.isEnableFilterRewrite() || Filters.filterMatchesNull(filterClause)) { return JoinFilterAnalysis.createNoPushdownFilterAnalysis(filterClause); } @@ -512,10 +235,7 @@ private static JoinFilterAnalysis analyzeJoinFilterClause( ); } - if (filterClause.supportsRequiredColumnRewrite() && doesRequiredColumnSetSupportDirectJoinFilterRewrite( - filterClause.getRequiredColumns(), - joinFilterPreAnalysis.getEquiconditions() - )) { + if (joinFilterPreAnalysis.getEquiconditions().doesFilterSupportDirectJoinFilterRewrite(filterClause)) { return rewriteFilterDirect( filterClause, joinFilterPreAnalysis, @@ -620,16 +340,15 @@ private static JoinFilterAnalysis rewriteOrFilter( boolean retainRhs = false; for (Filter filter : orFilter.getFilters()) { - if (!areSomeColumnsFromJoin(joinFilterPreAnalysis.getJoinableClauses(), filter.getRequiredColumns())) { + if (!joinFilterPreAnalysis.getJoinableClauses().areSomeColumnsFromJoin(filter.getRequiredColumns())) { newFilters.add(filter); continue; } JoinFilterAnalysis rewritten = null; - if (doesRequiredColumnSetSupportDirectJoinFilterRewrite( - filter.getRequiredColumns(), - joinFilterPreAnalysis.getEquiconditions() - )) { + if (joinFilterPreAnalysis.getEquiconditions() + .doesFilterSupportDirectJoinFilterRewrite(filter) + ) { rewritten = rewriteFilterDirect( filter, joinFilterPreAnalysis, @@ -687,7 +406,7 @@ private static JoinFilterAnalysis rewriteSelectorFilter( return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); } - if (!areSomeColumnsFromJoin(joinFilterPreAnalysis.getJoinableClauses(), selectorFilter.getRequiredColumns())) { + if (!joinFilterPreAnalysis.getJoinableClauses().areSomeColumnsFromJoin(selectorFilter.getRequiredColumns())) { return new JoinFilterAnalysis( false, selectorFilter, @@ -712,10 +431,20 @@ private static JoinFilterAnalysis rewriteSelectorFilter( return JoinFilterAnalysis.createNoPushdownFilterAnalysis(selectorFilter); } + Set newFilterValues = correlatedValues.get(); + // in nothing => match nothing + if (newFilterValues.isEmpty()) { + return new JoinFilterAnalysis( + true, + selectorFilter, + FalseFilter.instance() + ); + } + for (String correlatedBaseColumn : correlationAnalysis.getBaseColumns()) { Filter rewrittenFilter = new InDimFilter( correlatedBaseColumn, - correlatedValues.get(), + newFilterValues, null, null ).toFilter(); @@ -738,7 +467,7 @@ private static JoinFilterAnalysis rewriteSelectorFilter( Filter rewrittenFilter = new InDimFilter( pushDownVirtualColumn.getOutputName(), - correlatedValues.get(), + newFilterValues, null, null ).toFilter(); @@ -764,239 +493,6 @@ private static String getCorrelatedBaseExprVirtualColumnName(int counter) return PUSH_DOWN_VIRTUAL_COLUMN_NAME_BASE + counter; } - /** - * Helper method for rewriting filters on join table columns into filters on base table columns. - * - * @param filterColumn A join table column that we're filtering on - * @param filterValue The value to filter on - * @param correlatedJoinColumn A join table column that appears as the RHS of an equicondition, which we can correlate - * with a column on the base table - * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on - * - * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue - * Returns an empty set if we cannot determine the correlated values. - */ - private static Set getCorrelatedValuesForPushDown( - String filterColumn, - String filterValue, - String correlatedJoinColumn, - JoinableClause clauseForFilteredTable, - boolean enableRewriteValueColumnFilters, - long filterRewriteMaxSize - ) - { - String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length()); - String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length()); - - return clauseForFilteredTable.getJoinable().getCorrelatedColumnValues( - filterColumnNoPrefix, - filterValue, - correlatedColumnNoPrefix, - filterRewriteMaxSize, - enableRewriteValueColumnFilters - ); - } - - /** - * For each rhs column that appears in the equiconditions for a table's JoinableClause, - * we try to determine what base table columns are related to the rhs column through the total set of equiconditions. - * We do this by searching backwards through the chain of join equiconditions using the provided equicondition map. - * - * For example, suppose we have 3 tables, A,B,C, joined with the following conditions, where A is the base table: - * A.joinColumn == B.joinColumn - * B.joinColum == C.joinColumn - * - * We would determine that C.joinColumn is correlated with A.joinColumn: we first see that - * C.joinColumn is linked to B.joinColumn which in turn is linked to A.joinColumn - * - * Suppose we had the following join conditions instead: - * f(A.joinColumn) == B.joinColumn - * B.joinColum == C.joinColumn - * In this case, the JoinFilterColumnCorrelationAnalysis for C.joinColumn would be linked to f(A.joinColumn). - * - * Suppose we had the following join conditions instead: - * A.joinColumn == B.joinColumn - * f(B.joinColum) == C.joinColumn - * - * Because we cannot reverse the function f() applied to the second table B in all cases, - * we cannot relate C.joinColumn to A.joinColumn, and we would not generate a correlation for C.joinColumn - * - * @param joinableClauses List of joinable clauses for the query - * @param tablePrefix Prefix for a join table - * @param rhsRewriteCandidate RHS rewrite candidate that we find correlated base table columns for - * @param equiConditions Map of equiconditions, keyed by the right hand columns - * - * @return A list of correlatation analyses for the equicondition RHS columns that reside in the table associated with - * the tablePrefix - */ - private static Optional> findCorrelatedBaseTableColumns( - List joinableClauses, - String tablePrefix, - RhsRewriteCandidate rhsRewriteCandidate, - Map> equiConditions - ) - { - JoinableClause clauseForTablePrefix = rhsRewriteCandidate.getJoinableClause(); - JoinConditionAnalysis jca = clauseForTablePrefix.getCondition(); - - Set rhsColumns = new HashSet<>(); - if (rhsRewriteCandidate.isDirectRewrite()) { - // If we filter on a RHS join column, we only need to consider that column from the RHS side - rhsColumns.add(rhsRewriteCandidate.getRhsColumn()); - } else { - for (Equality eq : jca.getEquiConditions()) { - rhsColumns.add(tablePrefix + eq.getRightColumn()); - } - } - - Map correlations = new LinkedHashMap<>(); - - for (String rhsColumn : rhsColumns) { - Set correlatedBaseColumns = new HashSet<>(); - Set correlatedBaseExpressions = new HashSet<>(); - - getCorrelationForRHSColumn( - joinableClauses, - equiConditions, - rhsColumn, - correlatedBaseColumns, - correlatedBaseExpressions - ); - - if (correlatedBaseColumns.isEmpty() && correlatedBaseExpressions.isEmpty()) { - continue; - } - - correlations.put( - rhsColumn, - new JoinFilterColumnCorrelationAnalysis( - rhsColumn, - correlatedBaseColumns, - correlatedBaseExpressions - ) - ); - } - - if (correlations.size() == 0) { - return Optional.empty(); - } else { - return Optional.of(correlations); - } - } - - /** - * Helper method for {@link #findCorrelatedBaseTableColumns} that determines correlated base table columns - * and/or expressions for a single RHS column and adds them to the provided sets as it traverses the - * equicondition column relationships. - * - * @param equiConditions Map of equiconditions, keyed by the right hand columns - * @param rhsColumn RHS column to find base table correlations for - * @param correlatedBaseColumns Set of correlated base column names for the provided RHS column. Will be modified. - * @param correlatedBaseExpressions Set of correlated base column expressions for the provided RHS column. Will be - * modified. - */ - private static void getCorrelationForRHSColumn( - List joinableClauses, - Map> equiConditions, - String rhsColumn, - Set correlatedBaseColumns, - Set correlatedBaseExpressions - ) - { - String findMappingFor = rhsColumn; - Set lhsExprs = equiConditions.get(findMappingFor); - if (lhsExprs == null) { - return; - } - - for (Expr lhsExpr : lhsExprs) { - String identifier = lhsExpr.getBindingIfIdentifier(); - if (identifier == null) { - // We push down if the function only requires base table columns - Expr.BindingDetails bindingDetails = lhsExpr.analyzeInputs(); - Set requiredBindings = bindingDetails.getRequiredBindings(); - - if (areSomeColumnsFromJoin(joinableClauses, requiredBindings)) { - break; - } - correlatedBaseExpressions.add(lhsExpr); - } else { - // simple identifier, see if we can correlate it with a column on the base table - findMappingFor = identifier; - if (isColumnFromJoin(joinableClauses, identifier) == null) { - correlatedBaseColumns.add(findMappingFor); - } else { - getCorrelationForRHSColumn( - joinableClauses, - equiConditions, - findMappingFor, - correlatedBaseColumns, - correlatedBaseExpressions - ); - } - } - } - } - - /** - * Given a list of JoinFilterColumnCorrelationAnalysis, prune the list so that we only have one - * JoinFilterColumnCorrelationAnalysis for each unique combination of base columns. - * - * Suppose we have a join condition like the following, where A is the base table: - * A.joinColumn == B.joinColumn && A.joinColumn == B.joinColumn2 - * - * We only need to consider one correlation to A.joinColumn since B.joinColumn and B.joinColumn2 must - * have the same value in any row that matches the join condition. - * - * In the future this method could consider which column correlation should be preserved based on availability of - * indices and other heuristics. - * - * When push down of filters with LHS expressions in the join condition is supported, this method should also - * consider expressions. - * - * @param originalList Original list of column correlation analyses. - * - * @return Pruned list of column correlation analyses. - */ - private static List eliminateCorrelationDuplicates( - List originalList - ) - { - Map, JoinFilterColumnCorrelationAnalysis> uniquesMap = new HashMap<>(); - - for (JoinFilterColumnCorrelationAnalysis jca : originalList) { - Set mapKey = new HashSet<>(jca.getBaseColumns()); - for (Expr expr : jca.getBaseExpressions()) { - mapKey.add(expr.stringify()); - } - - uniquesMap.put(mapKey, jca); - } - - return new ArrayList<>(uniquesMap.values()); - } - - private static boolean filterMatchesNull(Filter filter) - { - ValueMatcher valueMatcher = filter.makeMatcher(ALL_NULL_COLUMN_SELECTOR_FACTORY); - return valueMatcher.matches(); - } - - @Nullable - private static JoinableClause isColumnFromJoin( - List joinableClauses, - String column - ) - { - for (JoinableClause joinableClause : joinableClauses) { - if (joinableClause.includesColumn(column)) { - return joinableClause; - } - } - - return null; - } - private static boolean isColumnFromPostJoinVirtualColumns( List postJoinVirtualColumns, String column @@ -1010,19 +506,6 @@ private static boolean isColumnFromPostJoinVirtualColumns( return false; } - private static boolean areSomeColumnsFromJoin( - List joinableClauses, - Collection columns - ) - { - for (String column : columns) { - if (isColumnFromJoin(joinableClauses, column) != null) { - return true; - } - } - return false; - } - private static boolean areSomeColumnsFromPostJoinVirtualColumns( List postJoinVirtualColumns, Collection columns @@ -1035,67 +518,4 @@ private static boolean areSomeColumnsFromPostJoinVirtualColumns( } return false; } - - private static void splitVirtualColumns( - List joinableClauses, - final VirtualColumns virtualColumns, - final List preJoinVirtualColumns, - final List postJoinVirtualColumns - ) - { - for (VirtualColumn virtualColumn : virtualColumns.getVirtualColumns()) { - if (areSomeColumnsFromJoin(joinableClauses, virtualColumn.requiredColumns())) { - postJoinVirtualColumns.add(virtualColumn); - } else { - preJoinVirtualColumns.add(virtualColumn); - } - } - } - - private static class RhsRewriteCandidate - { - private final boolean isDirectRewrite; - private final JoinableClause joinableClause; - private final String rhsColumn; - private final String valueForRewrite; - - public RhsRewriteCandidate( - JoinableClause joinableClause, - String rhsColumn, - String valueForRewrite, - boolean isDirectRewrite - ) - { - this.joinableClause = joinableClause; - this.rhsColumn = rhsColumn; - this.valueForRewrite = valueForRewrite; - this.isDirectRewrite = isDirectRewrite; - } - - public JoinableClause getJoinableClause() - { - return joinableClause; - } - - public String getRhsColumn() - { - return rhsColumn; - } - - public String getValueForRewrite() - { - return valueForRewrite; - } - - /** - * A direct rewrite occurs when we filter on an RHS column that is also part of a join equicondition. - * - * For example, if we have the filter (j.x = 'hello') and the join condition is (y = j.x), we can directly - * rewrite the j.x filter to (y = 'hello'). - */ - public boolean isDirectRewrite() - { - return isDirectRewrite; - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java index f978a8466a13..6071f404e499 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysis.java @@ -22,10 +22,12 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.Expr; +import javax.annotation.Nonnull; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -39,9 +41,9 @@ public class JoinFilterColumnCorrelationAnalysis { private final String joinColumn; - private final List baseColumns; - private final List baseExpressions; - private Map, Optional>> correlatedValuesMap; + @Nonnull private final List baseColumns; + @Nonnull private final List baseExpressions; + private final Map, Optional>> correlatedValuesMap; public JoinFilterColumnCorrelationAnalysis( String joinColumn, @@ -61,11 +63,13 @@ public String getJoinColumn() return joinColumn; } + @Nonnull public List getBaseColumns() { return baseColumns; } + @Nonnull public List getBaseExpressions() { return baseExpressions; @@ -80,4 +84,26 @@ public boolean supportsPushDown() { return !baseColumns.isEmpty() || !baseExpressions.isEmpty(); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JoinFilterColumnCorrelationAnalysis that = (JoinFilterColumnCorrelationAnalysis) o; + return Objects.equals(joinColumn, that.joinColumn) && + baseColumns.equals(that.baseColumns) && + baseExpressions.equals(that.baseExpressions) && + Objects.equals(correlatedValuesMap, that.correlatedValuesMap); + } + + @Override + public int hashCode() + { + return Objects.hash(joinColumn, baseColumns, baseExpressions, correlatedValuesMap); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterCorrelations.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterCorrelations.java new file mode 100644 index 000000000000..84fbccd8a572 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterCorrelations.java @@ -0,0 +1,407 @@ +/* + * 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.join.filter; + +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.join.Equality; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.rewrite.RhsRewriteCandidate; +import org.apache.druid.segment.join.filter.rewrite.RhsRewriteCandidates; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * A wrapper class for correlation analyses of different filters involved in the query. It contains: + * + * - A mapping of RHS filtering columns -> List, used for filter rewrites + * - A second mapping of RHS filtering columns -> List, used for direct filter rewrites + */ +public class JoinFilterCorrelations +{ + private final Map> correlationsByFilteringColumn; + private final Map> correlationsByDirectFilteringColumn; + + private JoinFilterCorrelations( + Map> correlationsByFilteringColumn, + Map> correlationsByDirectFilteringColumn + ) + { + this.correlationsByFilteringColumn = correlationsByFilteringColumn; + this.correlationsByDirectFilteringColumn = correlationsByDirectFilteringColumn; + } + + public Map> getCorrelationsByFilteringColumn() + { + return correlationsByFilteringColumn; + } + + public Map> getCorrelationsByDirectFilteringColumn() + { + return correlationsByDirectFilteringColumn; + } + + public static JoinFilterCorrelations computeJoinFilterCorrelations( + List normalizedJoinTableClauses, + Equiconditions equiconditions, + JoinableClauses joinableClauses, + boolean enableRewriteValueColumnFilters, + long filterRewriteMaxSize + ) + { + RhsRewriteCandidates rhsRewriteCandidates = + RhsRewriteCandidates.getRhsRewriteCandidates(normalizedJoinTableClauses, equiconditions, joinableClauses); + + // Build a map of RHS table prefix -> JoinFilterColumnCorrelationAnalysis based on the RHS rewrite candidates + Map>> correlationsByPrefix = new HashMap<>(); + Map> directRewriteCorrelations = new HashMap<>(); + for (RhsRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates.getRhsRewriteCandidates()) { + if (rhsRewriteCandidate.isDirectRewrite()) { + directRewriteCorrelations.computeIfAbsent( + rhsRewriteCandidate.getRhsColumn(), + c -> { + Optional> correlatedBaseTableColumns = + findCorrelatedBaseTableColumns( + joinableClauses, + c, + rhsRewriteCandidate, + equiconditions + ); + if (!correlatedBaseTableColumns.isPresent()) { + return Optional.empty(); + } else { + JoinFilterColumnCorrelationAnalysis baseColumnAnalysis = correlatedBaseTableColumns.get().get(c); + // for direct rewrites, there will only be one analysis keyed by the RHS column + assert (baseColumnAnalysis != null); + return Optional.of(correlatedBaseTableColumns.get().get(c)); + } + } + ); + } else { + correlationsByPrefix.computeIfAbsent( + rhsRewriteCandidate.getJoinableClause().getPrefix(), + p -> findCorrelatedBaseTableColumns( + joinableClauses, + p, + rhsRewriteCandidate, + equiconditions + ) + ); + } + } + + // Using the RHS table prefix -> JoinFilterColumnCorrelationAnalysis created in the previous step, + // build a map of rhsFilterColumn -> Pair(rhsFilterColumn, rhsFilterValue) -> correlatedValues for specific filter pair + // The Pair(rhsFilterColumn, rhsFilterValue) -> correlatedValues mappings are stored in the + // JoinFilterColumnCorrelationAnalysis objects, which are shared across all rhsFilterColumn entries that belong + // to the same RHS table. + // + // The value is a List instead of a single value because a table can be joined + // to another via multiple columns. + // (See JoinFilterAnalyzerTest.test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnRHS for an example) + Map> correlationsByFilteringColumn = new LinkedHashMap<>(); + Map> correlationsByDirectFilteringColumn = new LinkedHashMap<>(); + for (RhsRewriteCandidate rhsRewriteCandidate : rhsRewriteCandidates.getRhsRewriteCandidates()) { + if (rhsRewriteCandidate.isDirectRewrite()) { + List perColumnCorrelations = + correlationsByDirectFilteringColumn.computeIfAbsent( + rhsRewriteCandidate.getRhsColumn(), + (rhsCol) -> new ArrayList<>() + ); + perColumnCorrelations.add( + directRewriteCorrelations.get(rhsRewriteCandidate.getRhsColumn()).get() + ); + continue; + } + + Optional> correlationsForPrefix = correlationsByPrefix.get( + rhsRewriteCandidate.getJoinableClause().getPrefix() + ); + if (correlationsForPrefix.isPresent()) { + for (Map.Entry correlationForPrefix : correlationsForPrefix.get() + .entrySet()) { + List perColumnCorrelations = + correlationsByFilteringColumn.computeIfAbsent( + rhsRewriteCandidate.getRhsColumn(), + (rhsCol) -> new ArrayList<>() + ); + perColumnCorrelations.add(correlationForPrefix.getValue()); + correlationForPrefix.getValue().getCorrelatedValuesMap().computeIfAbsent( + Pair.of(rhsRewriteCandidate.getRhsColumn(), rhsRewriteCandidate.getValueForRewrite()), + (rhsVal) -> { + Optional> correlatedValues = getCorrelatedValuesForPushDown( + rhsRewriteCandidate.getRhsColumn(), + rhsRewriteCandidate.getValueForRewrite(), + correlationForPrefix.getValue().getJoinColumn(), + rhsRewriteCandidate.getJoinableClause(), + enableRewriteValueColumnFilters, + filterRewriteMaxSize + ); + return correlatedValues; + } + ); + } + } else { + correlationsByFilteringColumn.put(rhsRewriteCandidate.getRhsColumn(), null); + } + } + + // Go through each per-column analysis list and prune duplicates + for (Map.Entry> correlation : correlationsByFilteringColumn + .entrySet()) { + if (correlation.getValue() != null) { + List dedupList = eliminateCorrelationDuplicates( + correlation.getValue() + ); + correlationsByFilteringColumn.put(correlation.getKey(), dedupList); + } + } + for (Map.Entry> correlation : correlationsByDirectFilteringColumn + .entrySet()) { + if (correlation.getValue() != null) { + List dedupList = eliminateCorrelationDuplicates( + correlation.getValue() + ); + correlationsByDirectFilteringColumn.put(correlation.getKey(), dedupList); + } + } + return new JoinFilterCorrelations(correlationsByFilteringColumn, correlationsByDirectFilteringColumn); + } + + + /** + * Given a list of JoinFilterColumnCorrelationAnalysis, prune the list so that we only have one + * JoinFilterColumnCorrelationAnalysis for each unique combination of base columns. + *

+ * Suppose we have a join condition like the following, where A is the base table: + * A.joinColumn == B.joinColumn && A.joinColumn == B.joinColumn2 + *

+ * We only need to consider one correlation to A.joinColumn since B.joinColumn and B.joinColumn2 must + * have the same value in any row that matches the join condition. + *

+ * In the future this method could consider which column correlation should be preserved based on availability of + * indices and other heuristics. + *

+ * When push down of filters with LHS expressions in the join condition is supported, this method should also + * consider expressions. + * + * @param originalList Original list of column correlation analyses. + * @return Pruned list of column correlation analyses. + */ + private static List eliminateCorrelationDuplicates( + List originalList + ) + { + Map, JoinFilterColumnCorrelationAnalysis> uniquesMap = new HashMap<>(); + + for (JoinFilterColumnCorrelationAnalysis jca : originalList) { + Set mapKey = new HashSet<>(jca.getBaseColumns()); + for (Expr expr : jca.getBaseExpressions()) { + mapKey.add(expr.stringify()); + } + + uniquesMap.put(mapKey, jca); + } + + return new ArrayList<>(uniquesMap.values()); + } + + + /** + * Helper method for rewriting filters on join table columns into filters on base table columns. + * + * @param filterColumn A join table column that we're filtering on + * @param filterValue The value to filter on + * @param correlatedJoinColumn A join table column that appears as the RHS of an equicondition, which we can correlate + * with a column on the base table + * @param clauseForFilteredTable The joinable clause that corresponds to the join table being filtered on + * @return A list of values of the correlatedJoinColumn that appear in rows where filterColumn = filterValue + * Returns absent if we cannot determine the correlated values. + */ + private static Optional> getCorrelatedValuesForPushDown( + String filterColumn, + String filterValue, + String correlatedJoinColumn, + JoinableClause clauseForFilteredTable, + boolean enableRewriteValueColumnFilters, + long filterRewriteMaxSize + ) + { + String filterColumnNoPrefix = filterColumn.substring(clauseForFilteredTable.getPrefix().length()); + String correlatedColumnNoPrefix = correlatedJoinColumn.substring(clauseForFilteredTable.getPrefix().length()); + + return clauseForFilteredTable.getJoinable().getCorrelatedColumnValues( + filterColumnNoPrefix, + filterValue, + correlatedColumnNoPrefix, + filterRewriteMaxSize, + enableRewriteValueColumnFilters + ); + } + + /** + * For each rhs column that appears in the equiconditions for a table's JoinableClause, + * we try to determine what base table columns are related to the rhs column through the total set of equiconditions. + * We do this by searching backwards through the chain of join equiconditions using the provided equicondition map. + *

+ * For example, suppose we have 3 tables, A,B,C, joined with the following conditions, where A is the base table: + * A.joinColumn == B.joinColumn + * B.joinColum == C.joinColumnenableRewriteValueColumnFilters + *

+ * We would determine that C.joinColumn is correlated with A.joinColumn: we first see that + * C.joinColumn is linked to B.joinColumn which in turn is linked to A.joinColumn + *

+ * Suppose we had the following join conditions instead: + * f(A.joinColumn) == B.joinColumn + * B.joinColum == C.joinColumn + * In this case, the JoinFilterColumnCorrelationAnalysis for C.joinColumn would be linked to f(A.joinColumn). + *

+ * Suppose we had the following join conditions instead: + * A.joinColumn == B.joinColumn + * f(B.joinColum) == C.joinColumn + *

+ * Because we cannot reverse the function f() applied to the second table B in all cases, + * we cannot relate C.joinColumn to A.joinColumn, and we would not generate a correlation for C.joinColumn + * + * @param joinableClauses List of joinable clauses for the query + * @param tablePrefix Prefix for a join table + * @param rhsRewriteCandidate RHS rewrite candidate that we find correlated base table columns for + * @param equiConditions Map of equiconditions, keyed by the right hand columns + * @return A list of correlatation analyses for the equicondition RHS columns that reside in the table associated with + * the tablePrefix + */ + private static Optional> findCorrelatedBaseTableColumns( + JoinableClauses joinableClauses, + String tablePrefix, + RhsRewriteCandidate rhsRewriteCandidate, + Equiconditions equiConditions + ) + { + JoinableClause clauseForTablePrefix = rhsRewriteCandidate.getJoinableClause(); + JoinConditionAnalysis jca = clauseForTablePrefix.getCondition(); + + Set rhsColumns = new HashSet<>(); + if (rhsRewriteCandidate.isDirectRewrite()) { + // If we filter on a RHS join column, we only need to consider that column from the RHS side + rhsColumns.add(rhsRewriteCandidate.getRhsColumn()); + } else { + for (Equality eq : jca.getEquiConditions()) { + rhsColumns.add(tablePrefix + eq.getRightColumn()); + } + } + + Map correlations = new LinkedHashMap<>(); + + for (String rhsColumn : rhsColumns) { + Set correlatedBaseColumns = new HashSet<>(); + Set correlatedBaseExpressions = new HashSet<>(); + + getCorrelationForRHSColumn( + joinableClauses, + equiConditions, + rhsColumn, + correlatedBaseColumns, + correlatedBaseExpressions + ); + + if (correlatedBaseColumns.isEmpty() && correlatedBaseExpressions.isEmpty()) { + continue; + } + + correlations.put( + rhsColumn, + new JoinFilterColumnCorrelationAnalysis( + rhsColumn, + correlatedBaseColumns, + correlatedBaseExpressions + ) + ); + } + + if (correlations.size() == 0) { + return Optional.empty(); + } else { + return Optional.of(correlations); + } + } + + /** + * Helper method for {@link #findCorrelatedBaseTableColumns} that determines correlated base table columns + * and/or expressions for a single RHS column and adds them to the provided sets as it traverses the + * equicondition column relationships. + * + * @param equiConditions Map of equiconditions, keyed by the right hand columns + * @param rhsColumn RHS column to find base table correlations for + * @param correlatedBaseColumns Set of correlated base column names for the provided RHS column. Will be modified. + * @param correlatedBaseExpressions Set of correlated base column expressions for the provided RHS column. Will be + * modified. + */ + private static void getCorrelationForRHSColumn( + JoinableClauses joinableClauses, + Equiconditions equiConditions, + String rhsColumn, + Set correlatedBaseColumns, + Set correlatedBaseExpressions + ) + { + String findMappingFor = rhsColumn; + Set lhsExprs = equiConditions.getLhsExprs(findMappingFor); + if (lhsExprs == null) { + return; + } + + for (Expr lhsExpr : lhsExprs) { + String identifier = lhsExpr.getBindingIfIdentifier(); + if (identifier == null) { + // We push down if the function only requires base table columns + Expr.BindingDetails bindingDetails = lhsExpr.analyzeInputs(); + Set requiredBindings = bindingDetails.getRequiredBindings(); + + if (joinableClauses.areSomeColumnsFromJoin(requiredBindings)) { + break; + } + correlatedBaseExpressions.add(lhsExpr); + } else { + // simple identifier, see if we can correlate it with a column on the base table + findMappingFor = identifier; + if (joinableClauses.getColumnFromJoinIfExists(identifier) == null) { + correlatedBaseColumns.add(findMappingFor); + } else { + getCorrelationForRHSColumn( + joinableClauses, + equiConditions, + findMappingFor, + correlatedBaseColumns, + correlatedBaseExpressions + ); + } + } + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java index 991147b37f58..de842c3dde02 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinFilterPreAnalysis.java @@ -22,8 +22,15 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.query.filter.Filter; import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.join.Equality; import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -32,38 +39,32 @@ * A JoinFilterPreAnalysis contains filter push down/rewrite information that does not have per-segment dependencies. * This includes: * - The query's JoinableClauses list - * - The query's original filter (if any) + * - The original filter that an analysis was performed ons * - A list of filter clauses from the original filter's CNF representation that only reference the base table * - A list of filter clauses from the original filter's CNF representation that reference RHS join tables - * - A mapping of RHS filtering columns -> List, used for filter rewrites - * - A second mapping of RHS filtering columns -> List, used for direct filter rewrites * - A list of virtual columns that can only be computed post-join - * - Control flag booleans for whether filter push down and RHS rewrites are enabled. + * - The JoinFilterRewriteConfig that this pre-analysis is associated with. */ public class JoinFilterPreAnalysis { - private final List joinableClauses; + private final JoinableClauses joinableClauses; private final Filter originalFilter; private final List normalizedBaseTableClauses; private final List normalizedJoinTableClauses; - private final Map> correlationsByFilteringColumn; - private final Map> correlationsByDirectFilteringColumn; - private final boolean enableFilterPushDown; - private final boolean enableFilterRewrite; + private final JoinFilterCorrelations correlations; private final List postJoinVirtualColumns; - private final Map> equiconditions; + private final Equiconditions equiconditions; + private final JoinFilterRewriteConfig rewriteConfig; - public JoinFilterPreAnalysis( - final List joinableClauses, + private JoinFilterPreAnalysis( + final JoinableClauses joinableClauses, final Filter originalFilter, final List postJoinVirtualColumns, final List normalizedBaseTableClauses, final List normalizedJoinTableClauses, - final Map> correlationsByFilteringColumn, - final Map> correlationsByDirectFilteringColumn, - final boolean enableFilterPushDown, - final boolean enableFilterRewrite, - final Map> equiconditions + JoinFilterCorrelations correlations, + final Equiconditions equiconditions, + final JoinFilterRewriteConfig rewriteConfig ) { this.joinableClauses = joinableClauses; @@ -71,14 +72,12 @@ public JoinFilterPreAnalysis( this.postJoinVirtualColumns = postJoinVirtualColumns; this.normalizedBaseTableClauses = normalizedBaseTableClauses; this.normalizedJoinTableClauses = normalizedJoinTableClauses; - this.correlationsByFilteringColumn = correlationsByFilteringColumn; - this.correlationsByDirectFilteringColumn = correlationsByDirectFilteringColumn; - this.enableFilterPushDown = enableFilterPushDown; - this.enableFilterRewrite = enableFilterRewrite; + this.correlations = correlations; + this.rewriteConfig = rewriteConfig; this.equiconditions = equiconditions; } - public List getJoinableClauses() + public JoinableClauses getJoinableClauses() { return joinableClauses; } @@ -105,27 +104,106 @@ public List getNormalizedJoinTableClauses() public Map> getCorrelationsByFilteringColumn() { - return correlationsByFilteringColumn; + return correlations.getCorrelationsByFilteringColumn(); } public Map> getCorrelationsByDirectFilteringColumn() { - return correlationsByDirectFilteringColumn; + return correlations.getCorrelationsByDirectFilteringColumn(); } public boolean isEnableFilterPushDown() { - return enableFilterPushDown; + return rewriteConfig.isEnableFilterPushDown(); } public boolean isEnableFilterRewrite() { - return enableFilterRewrite; + return rewriteConfig.isEnableFilterRewrite(); } - public Map> getEquiconditions() + public Equiconditions getEquiconditions() { return equiconditions; } + + /** + * A Builder class to build {@link JoinFilterPreAnalysis} + */ + public static class Builder + { + @Nonnull private final JoinFilterRewriteConfig rewriteConfig; + @Nonnull private final JoinableClauses joinableClauses; + @Nullable private final Filter originalFilter; + @Nullable private List normalizedBaseTableClauses; + @Nullable private List normalizedJoinTableClauses; + @Nullable private JoinFilterCorrelations correlations; + @Nonnull private final List postJoinVirtualColumns; + @Nonnull private Equiconditions equiconditions = new Equiconditions(Collections.emptyMap()); + + public Builder( + @Nonnull JoinFilterRewriteConfig rewriteConfig, + @Nonnull JoinableClauses joinableClauses, + @Nullable Filter originalFilter, + @Nonnull List postJoinVirtualColumns + ) + { + this.rewriteConfig = rewriteConfig; + this.joinableClauses = joinableClauses; + this.originalFilter = originalFilter; + this.postJoinVirtualColumns = postJoinVirtualColumns; + } + + public Builder withNormalizedBaseTableClauses(List normalizedBaseTableClauses) + { + this.normalizedBaseTableClauses = normalizedBaseTableClauses; + return this; + } + + public Builder withNormalizedJoinTableClauses(List normalizedJoinTableClauses) + { + this.normalizedJoinTableClauses = normalizedJoinTableClauses; + return this; + } + + public Builder withCorrelations( + JoinFilterCorrelations correlations + ) + { + this.correlations = correlations; + return this; + } + + public Equiconditions computeEquiconditionsFromJoinableClauses() + { + Map> equiconditionsMap = new HashMap<>(); + for (JoinableClause clause : joinableClauses.getJoinableClauses()) { + for (Equality equality : clause.getCondition().getEquiConditions()) { + Set exprsForRhs = equiconditionsMap.computeIfAbsent( + clause.getPrefix() + equality.getRightColumn(), + (rhs) -> new HashSet<>() + ); + exprsForRhs.add(equality.getLeftExpr()); + } + } + this.equiconditions = new Equiconditions(equiconditionsMap); + return equiconditions; + } + + public JoinFilterPreAnalysis build() + { + return new JoinFilterPreAnalysis( + joinableClauses, + originalFilter, + postJoinVirtualColumns, + normalizedBaseTableClauses, + normalizedJoinTableClauses, + correlations, + equiconditions, + rewriteConfig + ); + } + + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/JoinableClauses.java b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinableClauses.java new file mode 100644 index 000000000000..44736ef8dd2f --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/JoinableClauses.java @@ -0,0 +1,136 @@ +/* + * 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.join.filter; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.query.planning.PreJoinableClause; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.join.Joinable; +import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.join.Joinables; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +public class JoinableClauses +{ + @Nonnull private final List joinableClauses; + + /** + * Builds a list of {@link JoinableClause} corresponding to a list of {@link PreJoinableClause}. This will call + * {@link JoinableFactory#build} on each one and therefore may be an expensive operation. + */ + public static JoinableClauses createClauses(final List clauses, final JoinableFactory joinableFactory) + { + // Since building a JoinableClause can be expensive, check for prefix conflicts before building + checkPreJoinableClausesForDuplicatesAndShadowing(clauses); + + List joinableClauses = clauses.stream().map(preJoinableClause -> { + final Optional joinable = joinableFactory.build( + preJoinableClause.getDataSource(), + preJoinableClause.getCondition() + ); + + return new JoinableClause( + preJoinableClause.getPrefix(), + joinable.orElseThrow(() -> new ISE("dataSource is not joinable: %s", preJoinableClause.getDataSource())), + preJoinableClause.getJoinType(), + preJoinableClause.getCondition() + ); + }).collect(Collectors.toList()); + return new JoinableClauses(joinableClauses); + } + + private JoinableClauses(@Nonnull List joinableClauses) + { + this.joinableClauses = joinableClauses; + } + + @Nonnull + public List getJoinableClauses() + { + return joinableClauses; + } + + public void splitVirtualColumns( + final VirtualColumns virtualColumns, + final List preJoinVirtualColumns, + final List postJoinVirtualColumns + ) + { + for (VirtualColumn virtualColumn : virtualColumns.getVirtualColumns()) { + if (areSomeColumnsFromJoin(virtualColumn.requiredColumns())) { + postJoinVirtualColumns.add(virtualColumn); + } else { + preJoinVirtualColumns.add(virtualColumn); + } + } + } + + public boolean areSomeColumnsFromJoin( + Collection columns + ) + { + for (String column : columns) { + if (getColumnFromJoinIfExists(column) != null) { + return true; + } + } + return false; + } + + @Nullable + public JoinableClause getColumnFromJoinIfExists( + String column + ) + { + for (JoinableClause joinableClause : joinableClauses) { + if (joinableClause.includesColumn(column)) { + return joinableClause; + } + } + + return null; + } + + private static void checkPreJoinableClausesForDuplicatesAndShadowing( + final List preJoinableClauses + ) + { + List prefixes = new ArrayList<>(); + for (PreJoinableClause clause : preJoinableClauses) { + prefixes.add(clause.getPrefix()); + } + + Joinables.checkPrefixesForDuplicatesAndShadowing(prefixes); + } + + public static JoinableClauses fromList(List clauses) + { + return new JoinableClauses(clauses); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.java b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.java new file mode 100644 index 000000000000..01d4b1d21f51 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterPreAnalysisGroup.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.segment.join.filter.rewrite; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; +import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.JoinableClauses; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * A JoinFilterPreAnalysisGroup holds all of the JoinFilterPreAnalysis objects for a given query and + * also stores the per-query parameters that control the filter rewrite operations (from the query context). + * + * The analyses map is keyed by (Filter, JoinableClause list, VirtualColumns): each Filter in the map belongs to a + * separate level of query (e.g. outer query, subquery level 1, etc.) + * + * If there is only a single Filter, then this class does not use the analyses map, instead of using a single reference + * for efficiency reasons. + */ +public class JoinFilterPreAnalysisGroup +{ + private final JoinFilterRewriteConfig joinFilterRewriteConfig; + private final Map analyses; + private final boolean isSingleLevelMode; + + /** + * Hashing and comparing filters can be expensive for large filters, so if we're only dealing with + * a single level of join query, then we can be more efficient by using a single reference instead of a map. + */ + private JoinFilterPreAnalysis preAnalysisForSingleLevelMode; + + public JoinFilterPreAnalysisGroup( + JoinFilterRewriteConfig joinFilterRewriteConfig, + boolean isSingleLevelMode + ) + { + this.joinFilterRewriteConfig = joinFilterRewriteConfig; + this.analyses = new HashMap<>(); + this.isSingleLevelMode = isSingleLevelMode; + } + + public boolean isSingleLevelMode() + { + return isSingleLevelMode; + } + + public JoinFilterPreAnalysis computeJoinFilterPreAnalysisIfAbsent( + Filter filter, + List clauses, + VirtualColumns virtualColumns + ) + { + if (isSingleLevelMode) { + preAnalysisForSingleLevelMode = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinableClauses.fromList(clauses), + virtualColumns, + filter, + joinFilterRewriteConfig + ); + return preAnalysisForSingleLevelMode; + } + + JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, virtualColumns); + return analyses.computeIfAbsent( + key, + (groupKey) -> { + return JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + JoinableClauses.fromList(clauses), + virtualColumns, + filter, + joinFilterRewriteConfig + ); + } + ); + } + + public JoinFilterPreAnalysis getAnalysis( + Filter filter, + VirtualColumns virtualColumns + ) + { + JoinFilterPreAnalysisGroupKey key = new JoinFilterPreAnalysisGroupKey(filter, virtualColumns); + return analyses.get(key); + } + + public JoinFilterPreAnalysis getPreAnalysisForSingleLevelMode() + { + return preAnalysisForSingleLevelMode; + } + + public static class JoinFilterPreAnalysisGroupKey + { + private final Filter filter; + private final VirtualColumns virtualColumns; + + public JoinFilterPreAnalysisGroupKey( + Filter filter, + VirtualColumns virtualColumns + ) + { + this.filter = filter; + this.virtualColumns = virtualColumns; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + JoinFilterPreAnalysisGroupKey that = (JoinFilterPreAnalysisGroupKey) o; + return Objects.equals(filter, that.filter) && + Objects.equals(virtualColumns, that.virtualColumns); + } + + @Override + public int hashCode() + { + return Objects.hash(filter, virtualColumns); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java new file mode 100644 index 000000000000..ed948bcc14f7 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/JoinFilterRewriteConfig.java @@ -0,0 +1,84 @@ +/* + * 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.join.filter.rewrite; + +/** + * A config class that holds properties that control how join filter rewrites behave. + */ +public class JoinFilterRewriteConfig +{ + /** + * Whether to enable filter push down optimizations to the base segment. + * In production this should generally be {@code QueryContexts.getEnableJoinFilterPushDown(query)}. + */ + private final boolean enableFilterPushDown; + + /** + * Whether to enable filter rewrite optimizations for RHS columns. + * In production this should generally be {@code QueryContexts.getEnableJoinFilterRewrite(query)}. + */ + private final boolean enableFilterRewrite; + + /** + * Whether to enable filter rewrite optimizations for RHS columns that are not key columns. + * In production this should generally be {@code QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query)}. + */ + private final boolean enableRewriteValueColumnFilters; + + /** + * The max allowed size of correlated value sets for RHS rewrites. In production + * This should generally be {@code QueryContexts.getJoinFilterRewriteMaxSize(query)}. + */ + private final long filterRewriteMaxSize; + + public JoinFilterRewriteConfig( + boolean enableFilterPushDown, + boolean enableFilterRewrite, + boolean enableRewriteValueColumnFilters, + long filterRewriteMaxSize + ) + { + this.enableFilterPushDown = enableFilterPushDown; + this.enableFilterRewrite = enableFilterRewrite; + this.enableRewriteValueColumnFilters = enableRewriteValueColumnFilters; + this.filterRewriteMaxSize = filterRewriteMaxSize; + } + + public boolean isEnableFilterPushDown() + { + return enableFilterPushDown; + } + + public boolean isEnableFilterRewrite() + { + return enableFilterRewrite; + } + + public boolean isEnableRewriteValueColumnFilters() + { + return enableRewriteValueColumnFilters; + } + + public long getFilterRewriteMaxSize() + { + return filterRewriteMaxSize; + } + +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidate.java b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidate.java new file mode 100644 index 000000000000..3a7a8df31916 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidate.java @@ -0,0 +1,101 @@ +/* + * 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.join.filter.rewrite; + +import org.apache.druid.segment.join.JoinableClause; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Objects; + +/** + * A candidate is an RHS column that appears in a filter, along with the value being filtered on, plus + * the joinable clause associated with the table that the RHS column is from. + */ +public class RhsRewriteCandidate +{ + private final boolean isDirectRewrite; + @Nonnull private final JoinableClause joinableClause; + private final String rhsColumn; + @Nullable private final String valueForRewrite; + + public RhsRewriteCandidate( + @Nonnull JoinableClause joinableClause, + String rhsColumn, + @Nullable String valueForRewrite, + boolean isDirectRewrite + ) + { + this.joinableClause = joinableClause; + this.rhsColumn = rhsColumn; + this.valueForRewrite = valueForRewrite; + this.isDirectRewrite = isDirectRewrite; + } + + @Nonnull + public JoinableClause getJoinableClause() + { + return joinableClause; + } + + public String getRhsColumn() + { + return rhsColumn; + } + + @Nullable + public String getValueForRewrite() + { + return valueForRewrite; + } + + /** + * A direct rewrite occurs when we filter on an RHS column that is also part of a join equicondition. + * + * For example, if we have the filter (j.x = 'hello') and the join condition is (y = j.x), we can directly + * rewrite the j.x filter to (y = 'hello'). + */ + public boolean isDirectRewrite() + { + return isDirectRewrite; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RhsRewriteCandidate that = (RhsRewriteCandidate) o; + return isDirectRewrite == that.isDirectRewrite && + joinableClause.equals(that.joinableClause) && + Objects.equals(rhsColumn, that.rhsColumn) && + Objects.equals(valueForRewrite, that.valueForRewrite); + } + + @Override + public int hashCode() + { + return Objects.hash(isDirectRewrite, joinableClause, rhsColumn, valueForRewrite); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidates.java b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidates.java new file mode 100644 index 000000000000..71c732fb1149 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidates.java @@ -0,0 +1,157 @@ +/* + * 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.join.filter.rewrite; + +import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.filter.OrFilter; +import org.apache.druid.segment.filter.SelectorFilter; +import org.apache.druid.segment.join.JoinableClause; +import org.apache.druid.segment.join.filter.Equiconditions; +import org.apache.druid.segment.join.filter.JoinableClauses; + +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +public class RhsRewriteCandidates +{ + private final Set rhsRewriteCandidates; + + private RhsRewriteCandidates(Set rhsRewriteCandidates) + { + this.rhsRewriteCandidates = rhsRewriteCandidates; + } + + public Set getRhsRewriteCandidates() + { + return rhsRewriteCandidates; + } + + /** + * Determine candidates for filter rewrites. + * A candidate is an RHS column that appears in a filter, along with the value being filtered on, plus + * the joinable clause associated with the table that the RHS column is from. + * + * These candidates are redued to filter rewrite correlations. + * + * @param normalizedJoinTableClauses + * @param equiconditions + * @param joinableClauses + * @return A set of candidates for filter rewrites. + */ + public static RhsRewriteCandidates getRhsRewriteCandidates( + List normalizedJoinTableClauses, + Equiconditions equiconditions, + JoinableClauses joinableClauses) + { + Set rhsRewriteCandidates = new LinkedHashSet<>(); + for (Filter orClause : normalizedJoinTableClauses) { + if (Filters.filterMatchesNull(orClause)) { + continue; + } + + if (orClause instanceof OrFilter) { + for (Filter subFilter : ((OrFilter) orClause).getFilters()) { + Optional rhsRewriteCandidate = determineRhsRewriteCandidatesForSingleFilter( + subFilter, + equiconditions, + joinableClauses + ); + + rhsRewriteCandidate.ifPresent(rhsRewriteCandidates::add); + } + continue; + } + + Optional rhsRewriteCandidate = determineRhsRewriteCandidatesForSingleFilter( + orClause, + equiconditions, + joinableClauses + ); + + rhsRewriteCandidate.ifPresent(rhsRewriteCandidates::add); + } + return new RhsRewriteCandidates(rhsRewriteCandidates); + } + + private static Optional determineRhsRewriteCandidatesForSingleFilter( + Filter orClause, + Equiconditions equiconditions, + JoinableClauses joinableClauses + ) + { + // Check if the filter clause is on the RHS join column. If so, we can rewrite the clause to filter on the + // LHS join column instead. + // Currently, we only support rewrites of filters that operate on a single column for simplicity. + if (equiconditions.doesFilterSupportDirectJoinFilterRewrite(orClause)) { + String reqColumn = orClause.getRequiredColumns().iterator().next(); + JoinableClause joinableClause = joinableClauses.getColumnFromJoinIfExists(reqColumn); + if (joinableClause != null) { + return Optional.of( + new RhsRewriteCandidate( + joinableClause, + reqColumn, + null, + true + ) + ); + } + } else if (orClause instanceof SelectorFilter) { + // this is a candidate for RHS filter rewrite, determine column correlations and correlated values + String reqColumn = ((SelectorFilter) orClause).getDimension(); + String reqValue = ((SelectorFilter) orClause).getValue(); + JoinableClause joinableClause = joinableClauses.getColumnFromJoinIfExists(reqColumn); + if (joinableClause != null) { + return Optional.of( + new RhsRewriteCandidate( + joinableClause, + reqColumn, + reqValue, + false + ) + ); + } + } + + return Optional.empty(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RhsRewriteCandidates that = (RhsRewriteCandidates) o; + return Objects.equals(rhsRewriteCandidates, that.rhsRewriteCandidates); + } + + @Override + public int hashCode() + { + return Objects.hash(rhsRewriteCandidates); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java index 624b6bc408bc..a97739b632e2 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinMatcher.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.Expr; +import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseFloatColumnValueSelector; @@ -71,9 +72,12 @@ public Supplier makeDimensionProcessor(DimensionSelector selector, boole if (row.size() == 1) { return selector.lookupName(row.get(0)); - } else { - // Multi-valued rows are not handled by the join system right now; treat them as nulls. + } else if (row.size() == 0) { return null; + } else { + // Multi-valued rows are not handled by the join system right now + // TODO: Remove when https://github.com/apache/druid/issues/9924 is done + throw new QueryUnsupportedException("Joining against a multi-value dimension is not supported."); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java index 353808dd6b33..9321a184ebbc 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/lookup/LookupJoinable.java @@ -31,8 +31,10 @@ import org.apache.druid.segment.join.Joinable; import javax.annotation.Nullable; +import java.io.Closeable; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.Set; public class LookupJoinable implements Joinable @@ -88,7 +90,7 @@ public JoinMatcher makeJoinMatcher( } @Override - public Set getCorrelatedColumnValues( + public Optional> getCorrelatedColumnValues( String searchColumnName, String searchColumnValue, String retrievalColumnName, @@ -97,7 +99,7 @@ public Set getCorrelatedColumnValues( ) { if (!ALL_COLUMNS.contains(searchColumnName) || !ALL_COLUMNS.contains(retrievalColumnName)) { - return ImmutableSet.of(); + return Optional.empty(); } Set correlatedValues; if (LookupColumnSelectorFactory.KEY_COLUMN.equals(searchColumnName)) { @@ -109,7 +111,7 @@ public Set getCorrelatedColumnValues( } } else { if (!allowNonKeyColumnSearch) { - return ImmutableSet.of(); + return Optional.empty(); } if (LookupColumnSelectorFactory.VALUE_COLUMN.equals(retrievalColumnName)) { // This should not happen in practice because the column to be joined on must be a key. @@ -120,6 +122,13 @@ public Set getCorrelatedColumnValues( correlatedValues = ImmutableSet.copyOf(extractor.unapply(searchColumnValue)); } } - return correlatedValues; + return Optional.of(correlatedValues); + } + + @Override + public Optional acquireReferences() + { + // nothing to close for lookup joinables, they are managed externally and have no per query accounting of usage + return Optional.of(() -> {}); } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java index cbc858112f30..b47214487299 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTable.java @@ -20,9 +20,11 @@ package org.apache.druid.segment.join.table; import it.unimi.dsi.fastutil.ints.IntList; +import org.apache.druid.segment.ReferenceCountedObject; import org.apache.druid.segment.column.RowSignature; import javax.annotation.Nullable; +import java.io.Closeable; import java.util.Set; /** @@ -30,8 +32,14 @@ * * The main user of this class is {@link IndexedTableJoinable}, and its main purpose is to participate in joins. */ -public interface IndexedTable +public interface IndexedTable extends ReferenceCountedObject, Closeable { + /** + * Returns the version of this table, used to compare against when loading a new version of the table + */ + @SuppressWarnings("unused") + String version(); + /** * Returns the columns of this table that have indexes. */ diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java index 7e6466d78288..00cb51ffffff 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableColumnSelectorFactory.java @@ -58,8 +58,9 @@ static ColumnCapabilities columnCapabilities(final IndexedTable table, final Str capabilities.setDictionaryValuesSorted(false); capabilities.setDictionaryValuesUnique(false); + capabilities.setHasMultipleValues(false); - return capabilities.setIsComplete(true); + return capabilities; } else { return null; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java index 685be4bb7673..455f4f12afca 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcher.java @@ -30,6 +30,7 @@ import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector; @@ -326,9 +327,12 @@ public Supplier makeDimensionProcessor(DimensionSelector selector, int dimensionId = row.get(0); IntList rowNumbers = getRowNumbers(selector, dimensionId); return rowNumbers.iterator(); - } else { - // Multi-valued rows are not handled by the join system right now; treat them as nulls. + } else if (row.size() == 0) { return IntIterators.EMPTY_ITERATOR; + } else { + // Multi-valued rows are not handled by the join system right now + // TODO: Remove when https://github.com/apache/druid/issues/9924 is done + throw new QueryUnsupportedException("Joining against a multi-value dimension is not supported."); } }; } else { @@ -341,9 +345,12 @@ public Supplier makeDimensionProcessor(DimensionSelector selector, int dimensionId = row.get(0); IntList rowNumbers = getAndCacheRowNumbers(selector, dimensionId); return rowNumbers.iterator(); - } else { - // Multi-valued rows are not handled by the join system right now; treat them as nulls. + } else if (row.size() == 0) { return IntIterators.EMPTY_ITERATOR; + } else { + // Multi-valued rows are not handled by the join system right now + // TODO: Remove when https://github.com/apache/druid/issues/9924 is done + throw new QueryUnsupportedException("Joining against a multi-value dimension is not supported."); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java index a661b5ad21ce..47166793ed20 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableJoinable.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.join.table; -import com.google.common.collect.ImmutableSet; import it.unimi.dsi.fastutil.ints.IntList; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnCapabilities; @@ -28,9 +27,11 @@ import org.apache.druid.segment.join.Joinable; import javax.annotation.Nullable; +import java.io.Closeable; import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Optional; import java.util.Set; public class IndexedTableJoinable implements Joinable @@ -82,7 +83,7 @@ public JoinMatcher makeJoinMatcher( } @Override - public Set getCorrelatedColumnValues( + public Optional> getCorrelatedColumnValues( String searchColumnName, String searchColumnValue, String retrievalColumnName, @@ -94,7 +95,7 @@ public Set getCorrelatedColumnValues( int correlatedColumnPosition = table.rowSignature().indexOf(retrievalColumnName); if (filterColumnPosition < 0 || correlatedColumnPosition < 0) { - return ImmutableSet.of(); + return Optional.empty(); } Set correlatedValues = new HashSet<>(); @@ -108,13 +109,13 @@ public Set getCorrelatedColumnValues( correlatedValues.add(correlatedDimVal); if (correlatedValues.size() > maxCorrelationSetSize) { - return ImmutableSet.of(); + return Optional.empty(); } } - return correlatedValues; + return Optional.of(correlatedValues); } else { if (!allowNonKeyColumnSearch) { - return ImmutableSet.of(); + return Optional.empty(); } IndexedTable.Reader dimNameReader = table.columnReader(filterColumnPosition); @@ -125,12 +126,18 @@ public Set getCorrelatedColumnValues( String correlatedDimVal = Objects.toString(correlatedColumnReader.read(i), null); correlatedValues.add(correlatedDimVal); if (correlatedValues.size() > maxCorrelationSetSize) { - return ImmutableSet.of(); + return Optional.empty(); } } } - return correlatedValues; + return Optional.of(correlatedValues); } } + + @Override + public Optional acquireReferences() + { + return table.acquireReferences(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java index 87ac7d48e17f..c18ec6a44b7f 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/RowBasedIndexedTable.java @@ -30,11 +30,13 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import java.io.Closeable; import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -50,12 +52,14 @@ public class RowBasedIndexedTable implements IndexedTable private final RowSignature rowSignature; private final List> columnFunctions; private final Set keyColumns; + private final String version; public RowBasedIndexedTable( final List table, final RowAdapter rowAdapter, final RowSignature rowSignature, - final Set keyColumns + final Set keyColumns, + final String version ) { this.table = table; @@ -63,6 +67,7 @@ public RowBasedIndexedTable( this.columnFunctions = rowSignature.getColumnNames().stream().map(rowAdapter::columnFunction).collect(Collectors.toList()); this.keyColumns = keyColumns; + this.version = version; if (new HashSet<>(keyColumns).size() != keyColumns.size()) { throw new ISE("keyColumns[%s] must not contain duplicates", keyColumns); @@ -106,6 +111,12 @@ public RowBasedIndexedTable( } } + @Override + public String version() + { + return version; + } + @Override public Set keyColumns() { @@ -163,4 +174,17 @@ public int numRows() { return table.size(); } + + @Override + public Optional acquireReferences() + { + // nothing to close by default, whatever loaded this thing (probably) lives on heap + return Optional.of(() -> {}); + } + + @Override + public void close() + { + // nothing to close + } } 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 3aeb32c1f262..9cc6dea06760 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 @@ -142,7 +142,7 @@ public String lookupName(final int id) @Override public boolean nameLookupPossibleInAdvance() { - return false; + return true; } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java index b9cfe4f03d69..269ac38429bb 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/QueryableIndexVectorColumnSelectorFactory.java @@ -85,7 +85,7 @@ public MultiValueDimensionVectorSelector makeMultiValueDimensionSelector(final D if (holder == null || !holder.getCapabilities().isDictionaryEncoded() || holder.getCapabilities().getType() != ValueType.STRING - || !holder.getCapabilities().hasMultipleValues()) { + || !holder.getCapabilities().hasMultipleValues().isMaybeTrue()) { throw new ISE( "Column[%s] is not a multi-value string column, do not ask for a multi-value selector", spec.getDimension() @@ -125,7 +125,7 @@ public SingleValueDimensionVectorSelector makeSingleValueDimensionSelector(final return NilVectorSelector.create(offset); } - if (holder.getCapabilities().hasMultipleValues()) { + if (holder.getCapabilities().hasMultipleValues().isMaybeTrue()) { // Asking for a single-value dimension selector on a multi-value column gets you an error. throw new ISE("Column[%s] is multi-value, do not ask for a single-value selector", spec.getDimension()); } diff --git a/processing/src/main/java/org/apache/druid/segment/vector/VectorColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/vector/VectorColumnSelectorFactory.java index bea845f7f5fb..ffa23cec433f 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/VectorColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/VectorColumnSelectorFactory.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.vector; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.column.ColumnCapabilities; import javax.annotation.Nullable; @@ -29,7 +30,7 @@ * * @see org.apache.druid.segment.ColumnSelectorFactory, the non-vectorized version. */ -public interface VectorColumnSelectorFactory +public interface VectorColumnSelectorFactory extends ColumnInspector { /** * Returns a {@link VectorSizeInspector} for the {@link VectorCursor} that generated this object. @@ -72,6 +73,7 @@ default int getMaxVectorSize() * * @return capabilities, or null if the column doesn't exist. */ + @Override @Nullable ColumnCapabilities getColumnCapabilities(String column); } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java index 5ae6987df88a..5ab4e4694a2b 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java @@ -154,8 +154,7 @@ public static ColumnValueSelector makeExprEvalSelector( } else if (capabilities != null && capabilities.getType() == ValueType.STRING && capabilities.isDictionaryEncoded() - && capabilities.isComplete() - && !capabilities.hasMultipleValues() + && !capabilities.hasMultipleValues().isMaybeTrue() && exprDetails.getArrayBindings().isEmpty()) { // Optimization for expressions that hit one scalar string column and nothing else. return new SingleStringInputCachingExpressionColumnValueSelector( @@ -227,7 +226,7 @@ public static DimensionSelector makeDimensionSelector( if (capabilities != null && capabilities.getType() == ValueType.STRING && capabilities.isDictionaryEncoded() - && capabilities.isComplete() + && !capabilities.hasMultipleValues().isUnknown() && !exprDetails.hasInputArrays() && !exprDetails.isOutputArray() ) { @@ -356,7 +355,7 @@ private static Expr.ObjectBinding createBindings( final ColumnCapabilities columnCapabilities = columnSelectorFactory .getColumnCapabilities(columnName); final ValueType nativeType = columnCapabilities != null ? columnCapabilities.getType() : null; - final boolean multiVal = columnCapabilities != null && columnCapabilities.hasMultipleValues(); + final boolean multiVal = columnCapabilities != null && columnCapabilities.hasMultipleValues().isTrue(); final Supplier supplier; if (nativeType == ValueType.FLOAT) { @@ -597,11 +596,11 @@ private static Pair, Set> examineColumnSelectorFactoryArrays for (String column : columns) { final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(column); if (capabilities != null) { - if (capabilities.hasMultipleValues()) { + if (capabilities.hasMultipleValues().isTrue()) { actualArrays.add(column); } else if ( - !capabilities.isComplete() && capabilities.getType().equals(ValueType.STRING) && + capabilities.hasMultipleValues().isMaybeTrue() && !exprDetails.getArrayBindings().contains(column) ) { unknownIfArrays.add(column); 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 482365152931..59b846fa9a99 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 @@ -131,10 +131,10 @@ public ColumnValueSelector makeColumnValueSelector(String columnName, ColumnS @Override public ColumnCapabilities capabilities(String columnName) { - // Note: Ideally we would only "setHasMultipleValues(true)" if the expression in question could potentially return - // multiple values. However, we don't currently have a good way of determining this, so to be safe we always - // set the flag. - return new ColumnCapabilitiesImpl().setType(outputType).setHasMultipleValues(true); + // Note: Ideally we would fill out additional information instead of leaving capabilities as 'unknown', e.g. examine + // if the expression in question could potentially return multiple values and anything else. However, we don't + // currently have a good way of determining this, so fill this out more once we do + return new ColumnCapabilitiesImpl().setType(outputType); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java index 090570db7acb..7c7f50f281bb 100644 --- a/processing/src/test/java/org/apache/druid/query/DataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/DataSourceTest.java @@ -99,5 +99,4 @@ public void testUnionDataSource() throws Exception final DataSource serde = JSON_MAPPER.readValue(JSON_MAPPER.writeValueAsString(dataSource), DataSource.class); Assert.assertEquals(dataSource, serde); } - } diff --git a/processing/src/test/java/org/apache/druid/query/GlobalTableDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/GlobalTableDataSourceTest.java new file mode 100644 index 000000000000..fea379015ad3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/GlobalTableDataSourceTest.java @@ -0,0 +1,67 @@ +/* + * 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.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +public class GlobalTableDataSourceTest +{ + private static final GlobalTableDataSource GLOBAL_TABLE_DATA_SOURCE = new GlobalTableDataSource("foo"); + + @Test + public void testEquals() + { + EqualsVerifier.forClass(GlobalTableDataSource.class) + .usingGetClass() + .withNonnullFields("name") + .verify(); + } + + @Test + public void testGlobalTableIsNotEqualsTable() + { + TableDataSource tbl = new TableDataSource(GLOBAL_TABLE_DATA_SOURCE.getName()); + Assert.assertNotEquals(GLOBAL_TABLE_DATA_SOURCE, tbl); + Assert.assertNotEquals(tbl, GLOBAL_TABLE_DATA_SOURCE); + } + + @Test + public void testIsGlobal() + { + Assert.assertTrue(GLOBAL_TABLE_DATA_SOURCE.isGlobal()); + } + + @Test + public void testSerde() throws JsonProcessingException + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final GlobalTableDataSource deserialized = (GlobalTableDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(GLOBAL_TABLE_DATA_SOURCE), + DataSource.class + ); + + Assert.assertEquals(GLOBAL_TABLE_DATA_SOURCE, deserialized); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java index 8aeb6f20f8dc..ce17b38ba46b 100644 --- a/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java +++ b/processing/src/test/java/org/apache/druid/query/SchemaEvolutionTest.java @@ -21,7 +21,10 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; import com.google.common.io.Closeables; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -35,6 +38,7 @@ import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.FloatSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; import org.apache.druid.query.expression.TestExprMacroTable; @@ -55,6 +59,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; import java.io.IOException; import java.util.HashMap; @@ -64,12 +69,18 @@ /** * Tests designed to exercise changing column types, adding columns, removing columns, etc. */ +@RunWith(JUnitParamsRunner.class) public class SchemaEvolutionTest { private static final String DATA_SOURCE = "foo"; private static final String TIMESTAMP_COLUMN = "t"; private static final double THIRTY_ONE_POINT_ONE = 31.1d; + public Object[] doVectorize() + { + return Lists.newArrayList(true, false).toArray(); + } + public static List> timeseriesResult(final Map map) { return ImmutableList.of(new Result<>(DateTimes.of("2000"), new TimeseriesResultValue((Map) map))); @@ -137,6 +148,8 @@ public static > List runQuery( @Before public void setUp() throws IOException { + NullHandling.initializeForTests(); + // Index1: c1 is a string, c2 nonexistent, "uniques" nonexistent index1 = IndexBuilder.create() .tmpDir(temporaryFolder.newFolder()) @@ -209,7 +222,8 @@ public void tearDown() throws IOException } @Test - public void testHyperUniqueEvolutionTimeseries() + @Parameters(method = "doVectorize") + public void testHyperUniqueEvolutionTimeseries(boolean doVectorize) { final TimeseriesQueryRunnerFactory factory = QueryRunnerTestHelper.newTimeseriesQueryRunnerFactory(); @@ -222,11 +236,12 @@ public void testHyperUniqueEvolutionTimeseries() new HyperUniquesAggregatorFactory("uniques", "uniques") ) ) + .context(ImmutableMap.of(QueryContexts.VECTORIZE_KEY, doVectorize)) .build(); // index1 has no "uniques" column Assert.assertEquals( - timeseriesResult(ImmutableMap.of("uniques", 0)), + timeseriesResult(ImmutableMap.of("uniques", 0d)), runQuery(query, factory, ImmutableList.of(index1)) ); @@ -238,7 +253,8 @@ public void testHyperUniqueEvolutionTimeseries() } @Test - public void testNumericEvolutionTimeseriesAggregation() + @Parameters(method = "doVectorize") + public void testNumericEvolutionTimeseriesAggregation(boolean doVectorize) { final TimeseriesQueryRunnerFactory factory = QueryRunnerTestHelper.newTimeseriesQueryRunnerFactory(); @@ -256,6 +272,7 @@ public void testNumericEvolutionTimeseriesAggregation() new DoubleSumAggregatorFactory("d", null, "c1 * 1", TestExprMacroTable.INSTANCE) ) ) + .context(ImmutableMap.of(QueryContexts.VECTORIZE_KEY, doVectorize)) .build(); // Only string(1) @@ -313,7 +330,8 @@ public void testNumericEvolutionTimeseriesAggregation() } @Test - public void testNumericEvolutionFiltering() + @Parameters(method = "doVectorize") + public void testNumericEvolutionFiltering(boolean doVectorize) { final TimeseriesQueryRunnerFactory factory = QueryRunnerTestHelper.newTimeseriesQueryRunnerFactory(); @@ -328,26 +346,28 @@ public void testNumericEvolutionFiltering() ImmutableList.of( new LongSumAggregatorFactory("a", "c1"), new DoubleSumAggregatorFactory("b", "c1"), + new FloatSumAggregatorFactory("d", "c1"), new CountAggregatorFactory("c") ) ) + .context(ImmutableMap.of(QueryContexts.VECTORIZE_KEY, doVectorize)) .build(); // Only string(1) -- which we can filter but not aggregate Assert.assertEquals( - timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.1, "c", 2L)), + timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.1, "c", 2L, "d", 19.1f)), runQuery(query, factory, ImmutableList.of(index1)) ); // Only long(2) -- which we can filter and aggregate Assert.assertEquals( - timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.0, "c", 2L)), + timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.0, "c", 2L, "d", 19.0f)), runQuery(query, factory, ImmutableList.of(index2)) ); // Only float(3) -- which we can't filter, but can aggregate Assert.assertEquals( - timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.1, "c", 2L)), + timeseriesResult(ImmutableMap.of("a", 19L, "b", 19.1, "c", 2L, "d", 19.1f)), runQuery(query, factory, ImmutableList.of(index3)) ); @@ -359,7 +379,9 @@ public void testNumericEvolutionFiltering() "b", NullHandling.defaultDoubleValue(), "c", - 0L + 0L, + "d", + NullHandling.defaultFloatValue() )), runQuery(query, factory, ImmutableList.of(index4)) ); @@ -369,7 +391,8 @@ public void testNumericEvolutionFiltering() timeseriesResult(ImmutableMap.of( "a", 57L, "b", 57.2, - "c", 6L + "c", 6L, + "d", 57.20000076293945 )), runQuery(query, factory, ImmutableList.of(index1, index2, index3, index4)) ); diff --git a/processing/src/test/java/org/apache/druid/query/TableDataSourceTest.java b/processing/src/test/java/org/apache/druid/query/TableDataSourceTest.java index ef50f3e45d9d..b5aeeb11bab4 100644 --- a/processing/src/test/java/org/apache/druid/query/TableDataSourceTest.java +++ b/processing/src/test/java/org/apache/druid/query/TableDataSourceTest.java @@ -36,6 +36,7 @@ public class TableDataSourceTest public ExpectedException expectedException = ExpectedException.none(); private final TableDataSource fooDataSource = new TableDataSource("foo"); + private final TableDataSource barDataSource = new TableDataSource("bar"); @Test public void test_getTableNames() @@ -85,31 +86,53 @@ public void test_withChildren_nonEmpty() @Test public void test_equals() { - EqualsVerifier.forClass(TableDataSource.class).withNonnullFields("name").verify(); + EqualsVerifier.forClass(TableDataSource.class).usingGetClass().withNonnullFields("name").verify(); } @Test - public void test_equals_legacy() + public void test_serde_roundTrip() throws Exception { - final LegacyDataSource legacyFoo = new LegacyDataSource("foo"); - final LegacyDataSource legacyBar = new LegacyDataSource("bar"); + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final TableDataSource deserialized = (TableDataSource) jsonMapper.readValue( + jsonMapper.writeValueAsString(fooDataSource), + DataSource.class + ); - Assert.assertEquals(legacyFoo, fooDataSource); - Assert.assertEquals(fooDataSource, legacyFoo); + Assert.assertEquals(fooDataSource, deserialized); + Assert.assertNotEquals(barDataSource, deserialized); + } - Assert.assertNotEquals(legacyBar, fooDataSource); - Assert.assertNotEquals(fooDataSource, legacyBar); + @Test + public void test_deserialize_fromObject() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final TableDataSource deserialized = (TableDataSource) jsonMapper.readValue( + "{\"type\":\"table\",\"name\":\"foo\"}", + DataSource.class + ); + + Assert.assertEquals(fooDataSource, deserialized); + Assert.assertNotEquals(barDataSource, deserialized); } @Test - public void test_serde() throws Exception + public void test_deserialize_fromString() throws Exception { final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); final TableDataSource deserialized = (TableDataSource) jsonMapper.readValue( - jsonMapper.writeValueAsString(fooDataSource), + "\"foo\"", DataSource.class ); Assert.assertEquals(fooDataSource, deserialized); + Assert.assertNotEquals(barDataSource, deserialized); + } + + @Test + public void test_serialize() throws Exception + { + final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); + final String s = jsonMapper.writeValueAsString(fooDataSource); + Assert.assertEquals("{\"type\":\"table\",\"name\":\"foo\"}", s); } } diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/StringColumnAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/StringColumnAggregationTest.java index b465925a8532..30419b04740f 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/StringColumnAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/StringColumnAggregationTest.java @@ -96,7 +96,7 @@ public void setup() throws Exception } aggregationTestHelper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - Collections.EMPTY_LIST, + Collections.emptyList(), new GroupByQueryConfig(), tempFolder ); @@ -231,7 +231,7 @@ public void testTimeseries() ) .build(); - Sequence seq = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(Collections.EMPTY_LIST, tempFolder) + Sequence seq = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper(Collections.emptyList(), tempFolder) .runQueryOnSegmentsObjs(segments, query); TimeseriesResultValue result = ((Result) Iterables.getOnlyElement(seq.toList())).getValue(); diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java index f9151923413d..421a457999d9 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/hyperloglog/HyperUniquesAggregatorFactoryTest.java @@ -30,6 +30,7 @@ import org.junit.Assert; import org.junit.Test; +import java.nio.ByteBuffer; import java.util.Comparator; import java.util.Random; @@ -173,6 +174,30 @@ public void testCompareToShouldBehaveConsistentlyWithEstimatedCardinalitiesEvenI } } + @Test + public void testEstimateCardinalityForZeroCardinality() + { + HyperLogLogCollector emptyHyperLogLogCollector = HyperUniquesBufferAggregator.doGet( + ByteBuffer.allocate(HyperLogLogCollector.getLatestNumBytesForDenseStorage()), + 0 + ); + + Assert.assertEquals(0L, HyperUniquesAggregatorFactory.estimateCardinality(null, true)); + Assert.assertEquals(0d, HyperUniquesAggregatorFactory.estimateCardinality(null, false)); + + Assert.assertEquals(0L, HyperUniquesAggregatorFactory.estimateCardinality(emptyHyperLogLogCollector, true)); + Assert.assertEquals(0d, HyperUniquesAggregatorFactory.estimateCardinality(emptyHyperLogLogCollector, false)); + + Assert.assertEquals( + HyperUniquesAggregatorFactory.estimateCardinality(emptyHyperLogLogCollector, true).getClass(), + HyperUniquesAggregatorFactory.estimateCardinality(null, true).getClass() + ); + Assert.assertEquals( + HyperUniquesAggregatorFactory.estimateCardinality(emptyHyperLogLogCollector, false).getClass(), + HyperUniquesAggregatorFactory.estimateCardinality(null, false).getClass() + ); + } + @Test public void testSerde() throws Exception { diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java index 2f0709e5bb45..cf1401993c39 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/mean/DoubleMeanAggregationTest.java @@ -21,12 +21,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; import org.apache.druid.data.input.Row; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.Druids; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.Result; import org.apache.druid.query.aggregation.AggregationTestHelper; import org.apache.druid.query.groupby.GroupByQuery; @@ -42,12 +47,19 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; import java.util.Collections; import java.util.List; +@RunWith(JUnitParamsRunner.class) public class DoubleMeanAggregationTest { + public Object[] doVectorize() + { + return Lists.newArrayList(true, false).toArray(); + } + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @@ -60,13 +72,13 @@ public DoubleMeanAggregationTest() { groupByQueryTestHelper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - Collections.EMPTY_LIST, + Collections.emptyList(), new GroupByQueryConfig(), tempFolder ); timeseriesQueryTestHelper = AggregationTestHelper.createTimeseriesQueryAggregationTestHelper( - Collections.EMPTY_LIST, + Collections.emptyList(), tempFolder ); @@ -77,7 +89,8 @@ public DoubleMeanAggregationTest() } @Test - public void testBufferAggretatorUsingGroupByQuery() throws Exception + @Parameters(method = "doVectorize") + public void testBufferAggretatorUsingGroupByQuery(boolean doVectorize) throws Exception { GroupByQuery query = new GroupByQuery.Builder() .setDataSource("test") @@ -88,6 +101,7 @@ public void testBufferAggretatorUsingGroupByQuery() throws Exception new DoubleMeanAggregatorFactory("meanOnString", SimpleTestIndex.SINGLE_VALUE_DOUBLE_AS_STRING_DIM), new DoubleMeanAggregatorFactory("meanOnMultiValue", SimpleTestIndex.MULTI_VALUE_DOUBLE_AS_STRING_DIM) ) + .setContext(ImmutableMap.of(QueryContexts.VECTORIZE_KEY, doVectorize)) .build(); // do json serialization and deserialization of query to ensure there are no serde issues @@ -103,7 +117,8 @@ public void testBufferAggretatorUsingGroupByQuery() throws Exception } @Test - public void testVectorAggretatorUsingGroupByQueryOnDoubleColumn() throws Exception + @Parameters(method = "doVectorize") + public void testVectorAggretatorUsingGroupByQueryOnDoubleColumn(boolean doVectorize) throws Exception { GroupByQuery query = new GroupByQuery.Builder() .setDataSource("test") @@ -112,7 +127,7 @@ public void testVectorAggretatorUsingGroupByQueryOnDoubleColumn() throws Excepti .setAggregatorSpecs( new DoubleMeanAggregatorFactory("meanOnDouble", SimpleTestIndex.DOUBLE_COL) ) - .setContext(Collections.singletonMap(GroupByQueryConfig.CTX_KEY_VECTORIZE, true)) + .setContext(Collections.singletonMap(GroupByQueryConfig.CTX_KEY_VECTORIZE, doVectorize)) .build(); // do json serialization and deserialization of query to ensure there are no serde issues @@ -126,7 +141,8 @@ public void testVectorAggretatorUsingGroupByQueryOnDoubleColumn() throws Excepti } @Test - public void testAggretatorUsingTimeseriesQuery() throws Exception + @Parameters(method = "doVectorize") + public void testAggretatorUsingTimeseriesQuery(boolean doVectorize) throws Exception { TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() .dataSource("test") @@ -143,6 +159,7 @@ public void testAggretatorUsingTimeseriesQuery() throws Exception SimpleTestIndex.MULTI_VALUE_DOUBLE_AS_STRING_DIM ) ) + .context(ImmutableMap.of(QueryContexts.VECTORIZE_KEY, doVectorize)) .build(); // do json serialization and deserialization of query to ensure there are no serde issues diff --git a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java index 0b63a720b840..aa5bd917bf13 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressMatchExprMacroTest.java @@ -23,7 +23,6 @@ import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprMacroTable; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import java.util.Arrays; @@ -42,12 +41,9 @@ public class IPv4AddressMatchExprMacroTest extends MacroTestBase private static final Expr SUBNET_10 = ExprEval.of("10.0.0.0/8").toExpr(); private static final Expr NOT_LITERAL = new NotLiteralExpr(null); - private IPv4AddressMatchExprMacro target; - - @Before - public void setUp() + public IPv4AddressMatchExprMacroTest() { - target = new IPv4AddressMatchExprMacro(); + super(new IPv4AddressMatchExprMacro()); } @Test @@ -55,7 +51,7 @@ public void testTooFewArgs() { expectException(IllegalArgumentException.class, "must have 2 arguments"); - target.apply(Collections.emptyList()); + apply(Collections.emptyList()); } @Test @@ -63,7 +59,7 @@ public void testTooManyArgs() { expectException(IllegalArgumentException.class, "must have 2 arguments"); - target.apply(Arrays.asList(IPV4, SUBNET_192_168, NOT_LITERAL)); + apply(Arrays.asList(IPV4, SUBNET_192_168, NOT_LITERAL)); } @Test @@ -71,7 +67,7 @@ public void testSubnetArgNotLiteral() { expectException(IllegalArgumentException.class, "subnet arg must be a literal"); - target.apply(Arrays.asList(IPV4, NOT_LITERAL)); + apply(Arrays.asList(IPV4, NOT_LITERAL)); } @Test @@ -80,7 +76,7 @@ public void testSubnetArgInvalid() expectException(IllegalArgumentException.class, "subnet arg has an invalid format"); Expr invalidSubnet = ExprEval.of("192.168.0.1/invalid").toExpr(); - target.apply(Arrays.asList(IPV4, invalidSubnet)); + apply(Arrays.asList(IPV4, invalidSubnet)); } @Test @@ -182,7 +178,7 @@ public void testInclusive() private boolean eval(Expr... args) { - Expr expr = target.apply(Arrays.asList(args)); + Expr expr = apply(Arrays.asList(args)); ExprEval eval = expr.eval(ExprUtils.nilBindings()); return eval.asBoolean(); } diff --git a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java index 2bf392141d51..0d70b2cce886 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressParseExprMacroTest.java @@ -23,7 +23,6 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import java.util.Arrays; @@ -35,12 +34,9 @@ public class IPv4AddressParseExprMacroTest extends MacroTestBase private static final long EXPECTED = 3232235521L; private static final Long NULL = NullHandling.replaceWithDefault() ? NullHandling.ZERO_LONG : null; - private IPv4AddressParseExprMacro target; - - @Before - public void setUp() + public IPv4AddressParseExprMacroTest() { - target = new IPv4AddressParseExprMacro(); + super(new IPv4AddressParseExprMacro()); } @Test @@ -48,7 +44,7 @@ public void testTooFewArgs() { expectException(IllegalArgumentException.class, "must have 1 argument"); - target.apply(Collections.emptyList()); + apply(Collections.emptyList()); } @Test @@ -56,7 +52,7 @@ public void testTooManyArgs() { expectException(IllegalArgumentException.class, "must have 1 argument"); - target.apply(Arrays.asList(VALID, VALID)); + apply(Arrays.asList(VALID, VALID)); } @Test @@ -154,7 +150,7 @@ public void testValidLongArg() private Object eval(Expr arg) { - Expr expr = target.apply(Collections.singletonList(arg)); + Expr expr = apply(Collections.singletonList(arg)); ExprEval eval = expr.eval(ExprUtils.nilBindings()); return eval.value(); } diff --git a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java index 602d00cfcc4e..1b4235b30686 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/IPv4AddressStringifyExprMacroTest.java @@ -23,7 +23,6 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.junit.Assert; -import org.junit.Before; import org.junit.Test; import java.util.Arrays; @@ -35,12 +34,9 @@ public class IPv4AddressStringifyExprMacroTest extends MacroTestBase private static final String EXPECTED = "192.168.0.1"; private static final String NULL = NullHandling.replaceWithDefault() ? "0.0.0.0" : null; - private IPv4AddressStringifyExprMacro target; - - @Before - public void setUp() + public IPv4AddressStringifyExprMacroTest() { - target = new IPv4AddressStringifyExprMacro(); + super(new IPv4AddressStringifyExprMacro()); } @Test @@ -48,7 +44,7 @@ public void testTooFewArgs() { expectException(IllegalArgumentException.class, "must have 1 argument"); - target.apply(Collections.emptyList()); + apply(Collections.emptyList()); } @Test @@ -56,7 +52,7 @@ public void testTooManyArgs() { expectException(IllegalArgumentException.class, "must have 1 argument"); - target.apply(Arrays.asList(VALID, VALID)); + apply(Arrays.asList(VALID, VALID)); } @Test @@ -150,7 +146,7 @@ public void testValidStringArgUnsignedInt() private Object eval(Expr arg) { - Expr expr = target.apply(Collections.singletonList(arg)); + Expr expr = apply(Collections.singletonList(arg)); ExprEval eval = expr.eval(ExprUtils.nilBindings()); return eval.value(); } diff --git a/processing/src/test/java/org/apache/druid/query/expression/MacroTestBase.java b/processing/src/test/java/org/apache/druid/query/expression/MacroTestBase.java index 38e607c2b8f4..7cf79073d0c9 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/MacroTestBase.java +++ b/processing/src/test/java/org/apache/druid/query/expression/MacroTestBase.java @@ -19,18 +19,80 @@ package org.apache.druid.query.expression; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.math.expr.Expr; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.math.expr.Parser; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; import org.junit.Rule; import org.junit.rules.ExpectedException; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; + public abstract class MacroTestBase extends InitializedNullHandlingTest { @Rule public ExpectedException expectedException = ExpectedException.none(); - void expectException(Class type, String message) + private final ExprMacroTable.ExprMacro macro; + + protected MacroTestBase(ExprMacroTable.ExprMacro macro) + { + this.macro = macro; + } + + protected void expectException(Class type, String message) { expectedException.expect(type); expectedException.expectMessage(message); } + + protected Expr apply(final List args) + { + return macro.apply(args); + } + + /** + * Evalutes {@code expr} using our macro. + * + * @param expression expression to evalute + * @param bindings bindings for evaluation + * + * @throws AssertionError if {@link ExprMacroTable.ExprMacro#apply} is not called on our macro during parsing + */ + protected ExprEval eval( + final String expression, + final Expr.ObjectBinding bindings + ) + { + // WrappedExprMacro allows us to confirm that our ExprMacro was actually called. + class WrappedExprMacro implements ExprMacroTable.ExprMacro + { + private final AtomicLong calls = new AtomicLong(); + + @Override + public String name() + { + return macro.name(); + } + + @Override + public Expr apply(List args) + { + calls.incrementAndGet(); + return macro.apply(args); + } + } + + final WrappedExprMacro wrappedMacro = new WrappedExprMacro(); + final GuiceExprMacroTable macroTable = new GuiceExprMacroTable(ImmutableSet.of(wrappedMacro)); + final Expr expr = Parser.parse(expression, macroTable); + + Assert.assertTrue("Calls made to macro.apply", wrappedMacro.calls.get() > 0); + + return expr.eval(bindings); + } } diff --git a/processing/src/test/java/org/apache/druid/query/expression/RegexpExtractExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/RegexpExtractExprMacroTest.java new file mode 100644 index 000000000000..2f811d788b4f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/expression/RegexpExtractExprMacroTest.java @@ -0,0 +1,141 @@ +/* + * 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.expression; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.Parser; +import org.junit.Assert; +import org.junit.Test; + +public class RegexpExtractExprMacroTest extends MacroTestBase +{ + public RegexpExtractExprMacroTest() + { + super(new RegexpExtractExprMacro()); + } + + @Test + public void testErrorZeroArguments() + { + expectException(IllegalArgumentException.class, "Function[regexp_extract] must have 2 to 3 arguments"); + eval("regexp_extract()", Parser.withMap(ImmutableMap.of())); + } + + @Test + public void testErrorFourArguments() + { + expectException(IllegalArgumentException.class, "Function[regexp_extract] must have 2 to 3 arguments"); + eval("regexp_extract('a', 'b', 'c', 'd')", Parser.withMap(ImmutableMap.of())); + } + + @Test + public void testMatch() + { + final ExprEval result = eval("regexp_extract(a, 'f(.o)')", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals("foo", result.value()); + } + + @Test + public void testMatchGroup0() + { + final ExprEval result = eval("regexp_extract(a, 'f(.o)', 0)", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals("foo", result.value()); + } + + @Test + public void testMatchGroup1() + { + final ExprEval result = eval("regexp_extract(a, 'f(.o)', 1)", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals("oo", result.value()); + } + + @Test + public void testMatchGroup2() + { + expectedException.expectMessage("No group 2"); + final ExprEval result = eval("regexp_extract(a, 'f(.o)', 2)", Parser.withMap(ImmutableMap.of("a", "foo"))); + } + + @Test + public void testNoMatch() + { + final ExprEval result = eval("regexp_extract(a, 'f(.x)')", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertNull(result.value()); + } + + @Test + public void testMatchInMiddle() + { + final ExprEval result = eval("regexp_extract(a, '.o$')", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals("oo", result.value()); + } + + @Test + public void testNullPattern() + { + if (NullHandling.sqlCompatible()) { + expectException(IllegalArgumentException.class, "Function[regexp_extract] pattern must be a string literal"); + } + + final ExprEval result = eval("regexp_extract(a, null)", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertNull(result.value()); + } + + @Test + public void testEmptyStringPattern() + { + final ExprEval result = eval("regexp_extract(a, '')", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals(NullHandling.emptyToNullIfNeeded(""), result.value()); + } + + @Test + public void testNumericPattern() + { + expectException(IllegalArgumentException.class, "Function[regexp_extract] pattern must be a string literal"); + eval("regexp_extract(a, 1)", Parser.withMap(ImmutableMap.of("a", "foo"))); + } + + @Test + public void testNonLiteralPattern() + { + expectException(IllegalArgumentException.class, "Function[regexp_extract] pattern must be a string literal"); + eval("regexp_extract(a, a)", Parser.withMap(ImmutableMap.of("a", "foo"))); + } + + @Test + public void testNullPatternOnNull() + { + if (NullHandling.sqlCompatible()) { + expectException(IllegalArgumentException.class, "Function[regexp_extract] pattern must be a string literal"); + } + + final ExprEval result = eval("regexp_extract(a, null)", Parser.withSuppliers(ImmutableMap.of("a", () -> null))); + Assert.assertNull(result.value()); + } + + @Test + public void testEmptyStringPatternOnNull() + { + final ExprEval result = eval("regexp_extract(a, '')", Parser.withSuppliers(ImmutableMap.of("a", () -> null))); + Assert.assertNull(result.value()); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/expression/RegexpLikeExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/RegexpLikeExprMacroTest.java new file mode 100644 index 000000000000..a6bdfb36a03a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/expression/RegexpLikeExprMacroTest.java @@ -0,0 +1,142 @@ +/* + * 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.expression; + +import com.google.common.collect.ImmutableMap; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.Parser; +import org.junit.Assert; +import org.junit.Test; + +public class RegexpLikeExprMacroTest extends MacroTestBase +{ + public RegexpLikeExprMacroTest() + { + super(new RegexpLikeExprMacro()); + } + + @Test + public void testErrorZeroArguments() + { + expectException(IllegalArgumentException.class, "Function[regexp_like] must have 2 arguments"); + eval("regexp_like()", Parser.withMap(ImmutableMap.of())); + } + + @Test + public void testErrorThreeArguments() + { + expectException(IllegalArgumentException.class, "Function[regexp_like] must have 2 arguments"); + eval("regexp_like('a', 'b', 'c')", Parser.withMap(ImmutableMap.of())); + } + + @Test + public void testMatch() + { + final ExprEval result = eval("regexp_like(a, 'f.o')", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals( + ExprEval.of(true, ExprType.LONG).value(), + result.value() + ); + } + + @Test + public void testNoMatch() + { + final ExprEval result = eval("regexp_like(a, 'f.x')", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals( + ExprEval.of(false, ExprType.LONG).value(), + result.value() + ); + } + + @Test + public void testNullPattern() + { + if (NullHandling.sqlCompatible()) { + expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a string literal"); + } + + final ExprEval result = eval("regexp_like(a, null)", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals( + ExprEval.of(true, ExprType.LONG).value(), + result.value() + ); + } + + @Test + public void testEmptyStringPattern() + { + final ExprEval result = eval("regexp_like(a, '')", Parser.withMap(ImmutableMap.of("a", "foo"))); + Assert.assertEquals( + ExprEval.of(true, ExprType.LONG).value(), + result.value() + ); + } + + @Test + public void testNullPatternOnEmptyString() + { + if (NullHandling.sqlCompatible()) { + expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a string literal"); + } + + final ExprEval result = eval("regexp_like(a, null)", Parser.withMap(ImmutableMap.of("a", ""))); + Assert.assertEquals( + ExprEval.of(true, ExprType.LONG).value(), + result.value() + ); + } + + @Test + public void testEmptyStringPatternOnEmptyString() + { + final ExprEval result = eval("regexp_like(a, '')", Parser.withMap(ImmutableMap.of("a", ""))); + Assert.assertEquals( + ExprEval.of(true, ExprType.LONG).value(), + result.value() + ); + } + + @Test + public void testNullPatternOnNull() + { + if (NullHandling.sqlCompatible()) { + expectException(IllegalArgumentException.class, "Function[regexp_like] pattern must be a string literal"); + } + + final ExprEval result = eval("regexp_like(a, null)", Parser.withSuppliers(ImmutableMap.of("a", () -> null))); + Assert.assertEquals( + ExprEval.of(true, ExprType.LONG).value(), + result.value() + ); + } + + @Test + public void testEmptyStringPatternOnNull() + { + final ExprEval result = eval("regexp_like(a, '')", Parser.withSuppliers(ImmutableMap.of("a", () -> null))); + Assert.assertEquals( + ExprEval.of(NullHandling.replaceWithDefault(), ExprType.LONG).value(), + result.value() + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java b/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java index 80ffdc2e0603..ab180646e585 100644 --- a/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java +++ b/processing/src/test/java/org/apache/druid/query/extraction/MapLookupExtractorTest.java @@ -64,7 +64,7 @@ public void testUnApply() ); } Assert.assertEquals(Sets.newHashSet(""), Sets.newHashSet(fn.unapply("empty_string"))); - Assert.assertEquals("not existing value returns empty list", Collections.EMPTY_LIST, fn.unapply("not There")); + Assert.assertEquals("not existing value returns empty list", Collections.emptyList(), fn.unapply("not There")); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java new file mode 100644 index 000000000000..ffa0253b2cf3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java @@ -0,0 +1,277 @@ +/* + * 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.filter.vector; + +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; +import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; +import org.apache.druid.segment.vector.VectorValueSelector; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.easymock.EasyMock; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNullHandlingTest +{ + private static final int VECTOR_SIZE = 128; + private static final int CURRENT_SIZE = 24; + private VectorValueSelector vectorValueSelector; + + @Before + public void setup() + { + vectorValueSelector = EasyMock.createMock(VectorValueSelector.class); + EasyMock.expect(vectorValueSelector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes(); + EasyMock.expect(vectorValueSelector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes(); + EasyMock.replay(vectorValueSelector); + } + + @Test + public void testFloat() + { + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeFloatProcessor(vectorValueSelector); + + Assert.assertTrue(matcherFactory instanceof FloatVectorValueMatcher); + + VectorValueMatcher matcher = matcherFactory.makeMatcher("2.0"); + Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize()); + + // in default mode, matching null produces a boolean matcher + VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null); + if (NullHandling.replaceWithDefault()) { + Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher); + } else { + Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher); + } + Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize()); + EasyMock.verify(vectorValueSelector); + } + + @Test + public void testDouble() + { + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeDoubleProcessor(vectorValueSelector); + + Assert.assertTrue(matcherFactory instanceof DoubleVectorValueMatcher); + + + VectorValueMatcher matcher = matcherFactory.makeMatcher("1.0"); + Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize()); + + // in default mode, matching null produces a boolean matcher + VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null); + if (NullHandling.replaceWithDefault()) { + Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher); + } else { + Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher); + } + Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize()); + EasyMock.verify(vectorValueSelector); + } + + @Test + public void testLong() + { + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeLongProcessor(vectorValueSelector); + + Assert.assertTrue(matcherFactory instanceof LongVectorValueMatcher); + + VectorValueMatcher matcher = matcherFactory.makeMatcher("1"); + Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize()); + + // in default mode, matching null produces a boolean matcher + VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher((String) null); + if (NullHandling.replaceWithDefault()) { + Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher); + } else { + Assert.assertFalse(booleanMatcher instanceof BooleanVectorValueMatcher); + } + Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize()); + EasyMock.verify(vectorValueSelector); + } + + @Test + public void testSingleValueString() + { + IdLookup lookup = EasyMock.createMock(IdLookup.class); + SingleValueDimensionVectorSelector selector = + EasyMock.createMock(SingleValueDimensionVectorSelector.class); + EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes(); + EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes(); + EasyMock.expect(selector.getValueCardinality()).andReturn(1024).anyTimes(); + EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(false).anyTimes(); + EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes(); + EasyMock.expect(lookup.lookupId("any value")).andReturn(1).anyTimes(); + EasyMock.expect(lookup.lookupId("another value")).andReturn(-1).anyTimes(); + EasyMock.replay(selector, lookup); + + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor(selector); + + Assert.assertTrue(matcherFactory instanceof SingleValueStringVectorValueMatcher); + + // value exists in column nonboolean matcher + VectorValueMatcher matcher = matcherFactory.makeMatcher("any value"); + Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize()); + + // value not exist in dictionary uses boolean matcher + VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher("another value"); + Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize()); + EasyMock.verify(selector, lookup); + } + + @Test + public void testSingleValueStringZeroCardinalityAlwaysBooleanMatcher() + { + // cardinality 0 has special path to always use boolean matcher + SingleValueDimensionVectorSelector selector = + EasyMock.createMock(SingleValueDimensionVectorSelector.class); + EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes(); + EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes(); + EasyMock.expect(selector.getValueCardinality()).andReturn(0).anyTimes(); + EasyMock.replay(selector); + + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor(selector); + + Assert.assertTrue(matcherFactory instanceof SingleValueStringVectorValueMatcher); + + VectorValueMatcher matcher = matcherFactory.makeMatcher("any value"); + Assert.assertTrue(matcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize()); + + // all are boolean with no valued column i guess + VectorValueMatcher anotherMatcher = matcherFactory.makeMatcher((String) null); + Assert.assertTrue(anotherMatcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, anotherMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, anotherMatcher.getCurrentVectorSize()); + EasyMock.verify(selector); + } + + @Test + public void testSingleValueStringOneCardinalityBooleanMatcherIfNullAndNameLookupPossible() + { + // single value string column with cardinality 1 and name lookup possible in advance uses boolean matcher for + // matches + SingleValueDimensionVectorSelector selector = + EasyMock.createMock(SingleValueDimensionVectorSelector.class); + EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes(); + EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes(); + EasyMock.expect(selector.getValueCardinality()).andReturn(1).anyTimes(); + EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(true).anyTimes(); + EasyMock.expect(selector.lookupName(0)).andReturn(null).anyTimes(); + EasyMock.replay(selector); + + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor(selector); + + Assert.assertTrue(matcherFactory instanceof SingleValueStringVectorValueMatcher); + + // false matcher + VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher("any value"); + Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize()); + + // true matcher + VectorValueMatcher anotherBooleanMatcher = matcherFactory.makeMatcher((String) null); + Assert.assertTrue(anotherBooleanMatcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, anotherBooleanMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, anotherBooleanMatcher.getCurrentVectorSize()); + EasyMock.verify(selector); + } + + @Test + public void testSingleValueStringOneCardinalityBooleanMatcherIfNullAndNameLookupNotPossible() + { + // if name lookup not possible in advance, use normal path, even if cardinality 1 + IdLookup lookup = EasyMock.createMock(IdLookup.class); + SingleValueDimensionVectorSelector selector = + EasyMock.createMock(SingleValueDimensionVectorSelector.class); + EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes(); + EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes(); + EasyMock.expect(selector.getValueCardinality()).andReturn(1).anyTimes(); + EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(false).anyTimes(); + EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes(); + EasyMock.expect(lookup.lookupId("any value")).andReturn(1).anyTimes(); + EasyMock.expect(lookup.lookupId(null)).andReturn(0).anyTimes(); + EasyMock.replay(selector, lookup); + + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor(selector); + + Assert.assertTrue(matcherFactory instanceof SingleValueStringVectorValueMatcher); + + VectorValueMatcher matcher = matcherFactory.makeMatcher("any value"); + Assert.assertFalse(matcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize()); + EasyMock.verify(selector, lookup); + } + + @Test + public void testMultiValueString() + { + IdLookup lookup = EasyMock.createMock(IdLookup.class); + MultiValueDimensionVectorSelector selector = EasyMock.createMock(MultiValueDimensionVectorSelector.class); + EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes(); + EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes(); + EasyMock.expect(selector.getValueCardinality()).andReturn(11).anyTimes(); + EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(false).anyTimes(); + EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes(); + EasyMock.expect(lookup.lookupId("any value")).andReturn(-1).anyTimes(); + EasyMock.expect(lookup.lookupId(null)).andReturn(0).anyTimes(); + EasyMock.replay(selector, lookup); + VectorValueMatcherFactory matcherFactory = + VectorValueMatcherColumnProcessorFactory.instance().makeMultiValueDimensionProcessor(selector); + + Assert.assertTrue(matcherFactory instanceof MultiValueStringVectorValueMatcher); + + VectorValueMatcher valueNotExistMatcher = matcherFactory.makeMatcher("any value"); + Assert.assertTrue(valueNotExistMatcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, valueNotExistMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, valueNotExistMatcher.getCurrentVectorSize()); + + VectorValueMatcher valueExistMatcher = matcherFactory.makeMatcher((String) null); + Assert.assertFalse(valueExistMatcher instanceof BooleanVectorValueMatcher); + Assert.assertEquals(VECTOR_SIZE, valueExistMatcher.getMaxVectorSize()); + Assert.assertEquals(CURRENT_SIZE, valueExistMatcher.getCurrentVectorSize()); + EasyMock.verify(selector, lookup); + } +} 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 221bc321b4aa..fa18a0abbe05 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 @@ -71,17 +71,6 @@ private static class TestBlockingPool extends CloseableDefaultBlockingPool take(final long timeout) - { - final ReferenceCountingResourceHolder holder = super.take(timeout); - final int poolSize = getPoolSize(); - if (minRemainBufferNum > poolSize) { - minRemainBufferNum = poolSize; - } - return holder; - } - @Override public List> takeBatch(final int maxElements, final long timeout) { 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 1b8519d68b49..31c438f8aab6 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 @@ -102,6 +102,7 @@ import org.apache.druid.query.filter.ExtractionDimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.JavaScriptDimFilter; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.RegexDimFilter; import org.apache.druid.query.filter.SearchQueryDimFilter; @@ -404,7 +405,11 @@ public ByteBuffer get() ); final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(strategySelector); final Closer closer = Closer.create(); - closer.register(bufferPool); + closer.register(() -> { + // Verify that all objects have been returned to the pool. + Assert.assertEquals(bufferPool.poolSize(), bufferPool.objectsCreatedCount()); + bufferPool.close(); + }); closer.register(mergeBufferPool); return Pair.of(new GroupByQueryRunnerFactory(strategySelector, toolChest), closer); } @@ -9874,7 +9879,7 @@ public void testGroupByLimitPushDownWithLongDimensionNotInLimitSpec() .setInterval(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) .setLimitSpec( new DefaultLimitSpec( - Collections.EMPTY_LIST, + Collections.emptyList(), 6 ) ).setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT) @@ -10435,6 +10440,35 @@ public void testTypeConversionWithMergingChainedExecutionRunner() TestHelper.assertExpectedObjects(expectedResults, results, "type-conversion"); } + @Test + public void testGroupByNoMatchingPrefilter() + { + GroupByQuery query = makeQueryBuilder() + .setDataSource(QueryRunnerTestHelper.DATA_SOURCE) + .setQuerySegmentSpec(QueryRunnerTestHelper.FIRST_TO_THIRD) + .setDimensions( + new DefaultDimensionSpec("quality", "quality") + ) + .setDimFilter(new SelectorDimFilter("market", "spot", null, null)) + .setAggregatorSpecs( + QueryRunnerTestHelper.ROWS_COUNT, + new FilteredAggregatorFactory( + new LongSumAggregatorFactory("index", "index"), + new NotDimFilter(new SelectorDimFilter("longNumericNull", null, null)) + ) + ) + .setGranularity(QueryRunnerTestHelper.DAY_GRAN) + .setLimit(1) + .build(); + + List expectedResults = ImmutableList.of( + makeRow(query, "2011-04-01", "quality", "automotive", "rows", 1L, "index", 135L) + ); + + Iterable results = GroupByQueryRunnerTestHelper.runQuery(factory, runner, query); + TestHelper.assertExpectedObjects(expectedResults, results, "groupBy"); + } + private static ResultRow makeRow(final GroupByQuery query, final String timestamp, final Object... vals) { return GroupByQueryRunnerTestHelper.createExpectedRow(query, timestamp, vals); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 8ddeb57f6237..9e7b45a58bac 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -235,4 +235,18 @@ public void testTimeseriesWithFilterOnNonExistentDimension() // Skip this test because the timeseries test expects a day that doesn't have a filter match to be filled in, // but group by just doesn't return a value if the filter doesn't match. } + + @Override + public void testTimeseriesWithTimestampResultFieldContextForArrayResponse() + { + // Skip this test because the timeseries test expects an extra column to be created (map from the timestamp_floor + // of the timestamp dimension) but group by doesn't do this. + } + + @Override + public void testTimeseriesWithTimestampResultFieldContextForMapResponse() + { + // Skip this test because the timeseries test expects an extra column to be created (map from the timestamp_floor + // of the timestamp dimension) but group by doesn't do this. + } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java index d71d70a1aac9..75a12a90cb6c 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/epinephelinae/GroupByQueryEngineV2Test.java @@ -47,8 +47,7 @@ public void testCanPushDownLimitForSegmentStringSelector() .setHasMultipleValues(false) .setDictionaryEncoded(true) .setDictionaryValuesSorted(true) - .setDictionaryValuesUnique(true) - .setIsComplete(true); + .setDictionaryValuesUnique(true); EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once(); EasyMock.replay(factory); Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); @@ -63,8 +62,7 @@ public void testCanPushDownLimitForIncrementalStringSelector() .setHasMultipleValues(false) .setDictionaryEncoded(false) .setDictionaryValuesSorted(false) - .setDictionaryValuesUnique(true) - .setIsComplete(true); + .setDictionaryValuesUnique(true); EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once(); EasyMock.replay(factory); Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); @@ -79,8 +77,7 @@ public void testCanPushDownLimitForExpressionStringSelector() .setHasMultipleValues(false) .setDictionaryEncoded(false) .setDictionaryValuesSorted(false) - .setDictionaryValuesUnique(false) - .setIsComplete(true); + .setDictionaryValuesUnique(false); EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once(); EasyMock.replay(factory); Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); @@ -95,8 +92,7 @@ public void testCanPushDownLimitForJoinStringSelector() .setHasMultipleValues(false) .setDictionaryEncoded(true) .setDictionaryValuesSorted(false) - .setDictionaryValuesUnique(false) - .setIsComplete(true); + .setDictionaryValuesUnique(false); EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once(); EasyMock.replay(factory); Assert.assertFalse(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); @@ -111,8 +107,7 @@ public void testCanPushDownLimitForNumericSelector() .setHasMultipleValues(false) .setDictionaryEncoded(false) .setDictionaryValuesSorted(false) - .setDictionaryValuesUnique(false) - .setIsComplete(true); + .setDictionaryValuesUnique(false); EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).anyTimes(); EasyMock.replay(factory); Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); @@ -131,8 +126,7 @@ public void testCanPushDownLimitForComplexSelector() .setHasMultipleValues(false) .setDictionaryEncoded(false) .setDictionaryValuesSorted(false) - .setDictionaryValuesUnique(false) - .setIsComplete(true); + .setDictionaryValuesUnique(false); EasyMock.expect(factory.getColumnCapabilities(DIM)).andReturn(capabilities).once(); EasyMock.replay(factory); Assert.assertTrue(GroupByQueryEngineV2.canPushDownLimit(factory, DIM)); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparatorTest.java b/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparatorTest.java index 135328df0035..46ba97583951 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/having/HavingSpecMetricComparatorTest.java @@ -41,4 +41,64 @@ public void testLongRegex() Assert.assertFalse(HavingSpecMetricComparator.LONG_PAT.matcher("").matches()); Assert.assertFalse(HavingSpecMetricComparator.LONG_PAT.matcher("xyz").matches()); } + + @Test + public void testCompareDoubleToLongWithNanReturns1() + { + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.NaN, 1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.NaN, -1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.NaN, Long.MAX_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.NaN, Long.MIN_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.NaN, 0L)); + + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.NaN, 1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.NaN, -1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.NaN, Long.MAX_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.NaN, Long.MIN_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.NaN, 0L)); + } + + @Test + public void testCompareDoubleToLongWithInfinityReturns1() + { + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.POSITIVE_INFINITY, 1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.POSITIVE_INFINITY, -1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.POSITIVE_INFINITY, Long.MAX_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.POSITIVE_INFINITY, Long.MIN_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Double.POSITIVE_INFINITY, 0L)); + + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.POSITIVE_INFINITY, 1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.POSITIVE_INFINITY, -1)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.POSITIVE_INFINITY, Long.MAX_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.POSITIVE_INFINITY, Long.MIN_VALUE)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(Float.POSITIVE_INFINITY, 0L)); + } + + @Test + public void testCompareDoubleToLongWithInfinityReturnsNegative1() + { + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Double.NEGATIVE_INFINITY, 1)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Double.NEGATIVE_INFINITY, -1)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Double.NEGATIVE_INFINITY, Long.MAX_VALUE)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Double.NEGATIVE_INFINITY, Long.MIN_VALUE)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Double.NEGATIVE_INFINITY, 0L)); + + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Float.NEGATIVE_INFINITY, 1)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Float.NEGATIVE_INFINITY, -1)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Float.NEGATIVE_INFINITY, Long.MAX_VALUE)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Float.NEGATIVE_INFINITY, Long.MIN_VALUE)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(Float.NEGATIVE_INFINITY, 0L)); + } + + @Test + public void testCompareDoubleToLongWithNumbers() + { + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong(1 + 1e-6, 1)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong(1 - 1e-6, 1)); + Assert.assertEquals(0, HavingSpecMetricComparator.compareDoubleToLong(10D, 10)); + Assert.assertEquals(0, HavingSpecMetricComparator.compareDoubleToLong(0D, 0)); + Assert.assertEquals(0, HavingSpecMetricComparator.compareDoubleToLong(-0D, 0)); + Assert.assertEquals(1, HavingSpecMetricComparator.compareDoubleToLong((double) Long.MAX_VALUE + 1, Long.MAX_VALUE)); + Assert.assertEquals(-1, HavingSpecMetricComparator.compareDoubleToLong((double) Long.MIN_VALUE - 1, Long.MIN_VALUE)); + } } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/orderby/TopNSequenceTest.java b/processing/src/test/java/org/apache/druid/query/groupby/orderby/TopNSequenceTest.java index 4343df5d3793..24238af0ab83 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/orderby/TopNSequenceTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/orderby/TopNSequenceTest.java @@ -42,7 +42,7 @@ public class TopNSequenceTest private static final Ordering ASC = Ordering.natural(); private static final Ordering DESC = Ordering.natural().reverse(); - private static final List EMPTY = Collections.EMPTY_LIST; + private static final List EMPTY = Collections.emptyList(); private static final List SINGLE = Collections.singletonList("a"); private static final List RAW_ASC = Lists.newArrayList(Splitter.fixedLength(1).split("abcdefghijk")); private static final List RAW_DESC = Lists.newArrayList(Splitter.fixedLength(1).split("kjihgfedcba")); 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 ce6035142374..3ca72aa5a698 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 @@ -137,9 +137,8 @@ public void test_asStorageAdapter_getColumnCapabilitiesK() // Note: the "k" 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 // running on top of lookups can take advantage of singly-valued optimizations. - Assert.assertFalse(capabilities.hasMultipleValues()); + Assert.assertTrue(capabilities.hasMultipleValues().isUnknown()); Assert.assertFalse(capabilities.isDictionaryEncoded()); - Assert.assertFalse(capabilities.isComplete()); } @Test @@ -151,9 +150,8 @@ public void test_asStorageAdapter_getColumnCapabilitiesV() // reporting complete single-valued capabilities. It would be good to change this in the future, so query engines // running on top of lookups can take advantage of singly-valued optimizations. Assert.assertEquals(ValueType.STRING, capabilities.getType()); - Assert.assertFalse(capabilities.hasMultipleValues()); + Assert.assertTrue(capabilities.hasMultipleValues().isUnknown()); Assert.assertFalse(capabilities.isDictionaryEncoded()); - Assert.assertFalse(capabilities.isComplete()); } @Test 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 4f3a81dbb208..66df87904daa 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 @@ -20,11 +20,11 @@ package org.apache.druid.query.metadata; import org.apache.druid.data.input.impl.DimensionSchema; -import org.apache.druid.query.LegacyDataSource; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.metadata.metadata.ColumnAnalysis; import org.apache.druid.query.metadata.metadata.SegmentAnalysis; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; @@ -190,7 +190,7 @@ private List getSegmentAnalysises(Segment index, EnumSet constructorFeeder() } @Test - public void testSortAndLimitScanResultValues() + public void testSortAndLimitScanResultValues() throws IOException { List srvs = new ArrayList<>(numElements); List expectedEventTimestamps = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java index 3f6e59b7f493..b7bc426f8f9b 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChestTest.java @@ -56,6 +56,7 @@ @RunWith(Parameterized.class) public class TimeseriesQueryQueryToolChestTest { + private static final String TIMESTAMP_RESULT_FIELD_NAME = "d0"; private static final TimeseriesQueryQueryToolChest TOOL_CHEST = new TimeseriesQueryQueryToolChest(null); @BeforeClass @@ -364,7 +365,7 @@ public void testResultLevelCacheKeyWithGrandTotal() } @Test - public void testResultArraySignature() + public void testResultArraySignatureWithoutTimestampResultField() { final TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() @@ -388,6 +389,33 @@ public void testResultArraySignature() ); } + @Test + public void testResultArraySignatureWithTimestampResultField() + { + final TimeseriesQuery query = + Druids.newTimeseriesQueryBuilder() + .dataSource("dummy") + .intervals("2000/3000") + .descending(descending) + .granularity(Granularities.HOUR) + .aggregators(QueryRunnerTestHelper.COMMON_DOUBLE_AGGREGATORS) + .postAggregators(QueryRunnerTestHelper.CONSTANT) + .context(ImmutableMap.of(TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, TIMESTAMP_RESULT_FIELD_NAME)) + .build(); + + Assert.assertEquals( + RowSignature.builder() + .addTimeColumn() + .add(TIMESTAMP_RESULT_FIELD_NAME, ValueType.LONG) + .add("rows", ValueType.LONG) + .add("index", ValueType.DOUBLE) + .add("uniques", null) + .add("const", null) + .build(), + TOOL_CHEST.resultArraySignature(query) + ); + } + @Test public void testResultsAsArrays() { diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java index 4c0154b120c3..99989c01d8b4 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -31,6 +31,7 @@ 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.java.util.common.guava.Sequence; import org.apache.druid.query.Druids; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.QueryPlus; @@ -59,6 +60,7 @@ import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -87,6 +89,7 @@ @RunWith(Parameterized.class) public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest { + private static final String TIMESTAMP_RESULT_FIELD_NAME = "d0"; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -2471,6 +2474,246 @@ public void testTimeseriesWithBoundFilter1() TestHelper.assertExpectedResults(expectedResults, results); } + @Test + public void testTimeseriesWithTimestampResultFieldContextForArrayResponse() + { + Granularity gran = Granularities.DAY; + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .granularity(gran) + .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .aggregators( + QueryRunnerTestHelper.ROWS_COUNT, + QueryRunnerTestHelper.INDEX_DOUBLE_SUM, + QueryRunnerTestHelper.QUALITY_UNIQUES + ) + .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) + .descending(descending) + .context(ImmutableMap.of( + TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, TIMESTAMP_RESULT_FIELD_NAME + )) + .build(); + + Assert.assertEquals(TIMESTAMP_RESULT_FIELD_NAME, query.getTimestampResultField()); + + QueryToolChest, TimeseriesQuery> toolChest = new TimeseriesQueryQueryToolChest(); + + RowSignature rowSignature = toolChest.resultArraySignature(query); + Assert.assertNotNull(rowSignature); + List columnNames = rowSignature.getColumnNames(); + Assert.assertNotNull(columnNames); + Assert.assertEquals(6, columnNames.size()); + Assert.assertEquals("__time", columnNames.get(0)); + Assert.assertEquals(TIMESTAMP_RESULT_FIELD_NAME, columnNames.get(1)); + Assert.assertEquals("rows", columnNames.get(2)); + Assert.assertEquals("index", columnNames.get(3)); + Assert.assertEquals("uniques", columnNames.get(4)); + Assert.assertEquals("addRowsIndexConstant", columnNames.get(5)); + + Sequence> results = runner.run(QueryPlus.wrap(query)); + Sequence resultsAsArrays = toolChest.resultsAsArrays(query, results); + + Assert.assertNotNull(resultsAsArrays); + + final String[] expectedIndex = descending ? + QueryRunnerTestHelper.EXPECTED_FULL_ON_INDEX_VALUES_DESC : + QueryRunnerTestHelper.EXPECTED_FULL_ON_INDEX_VALUES; + + final Long expectedLast = descending ? + QueryRunnerTestHelper.EARLIEST.getMillis() : + QueryRunnerTestHelper.LAST.getMillis(); + + int count = 0; + Object[] lastResult = null; + for (Object[] result : resultsAsArrays.toList()) { + Long current = (Long) result[0]; + Assert.assertFalse( + StringUtils.format("Timestamp[%s] > expectedLast[%s]", current, expectedLast), + descending ? current < expectedLast : current > expectedLast + ); + + Assert.assertEquals( + (Long) result[1], + current, + 0 + ); + + Assert.assertEquals( + QueryRunnerTestHelper.SKIPPED_DAY.getMillis() == current ? (Long) 0L : (Long) 13L, + result[2] + ); + + if (QueryRunnerTestHelper.SKIPPED_DAY.getMillis() != current) { + Assert.assertEquals( + Doubles.tryParse(expectedIndex[count]).doubleValue(), + (Double) result[3], + (Double) result[3] * 1e-6 + ); + Assert.assertEquals( + (Double) result[4], + 9.0d, + 0.02 + ); + Assert.assertEquals( + new Double(expectedIndex[count]) + 13L + 1L, + (Double) result[5], + (Double) result[5] * 1e-6 + ); + } else { + if (NullHandling.replaceWithDefault()) { + Assert.assertEquals( + 0.0D, + (Double) result[3], + (Double) result[3] * 1e-6 + ); + Assert.assertEquals( + 0.0D, + (Double) result[4], + 0.02 + ); + Assert.assertEquals( + new Double(expectedIndex[count]) + 1L, + (Double) result[5], + (Double) result[5] * 1e-6 + ); + } else { + Assert.assertNull( + result[3] + ); + Assert.assertEquals( + (Double) result[4], + 0.0, + 0.02 + ); + Assert.assertNull( + result[5] + ); + } + } + + lastResult = result; + ++count; + } + Assert.assertEquals(expectedLast, lastResult[0]); + } + + @Test + public void testTimeseriesWithTimestampResultFieldContextForMapResponse() + { + Granularity gran = Granularities.DAY; + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .granularity(gran) + .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC) + .aggregators( + QueryRunnerTestHelper.ROWS_COUNT, + QueryRunnerTestHelper.INDEX_DOUBLE_SUM, + QueryRunnerTestHelper.QUALITY_UNIQUES + ) + .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) + .descending(descending) + .context(ImmutableMap.of( + TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, TIMESTAMP_RESULT_FIELD_NAME + )) + .build(); + + Assert.assertEquals(TIMESTAMP_RESULT_FIELD_NAME, query.getTimestampResultField()); + + Iterable> results = runner.run(QueryPlus.wrap(query)).toList(); + + final String[] expectedIndex = descending ? + QueryRunnerTestHelper.EXPECTED_FULL_ON_INDEX_VALUES_DESC : + QueryRunnerTestHelper.EXPECTED_FULL_ON_INDEX_VALUES; + + final DateTime expectedLast = descending ? + QueryRunnerTestHelper.EARLIEST : + QueryRunnerTestHelper.LAST; + + int count = 0; + Result lastResult = null; + for (Result result : results) { + DateTime current = result.getTimestamp(); + Assert.assertFalse( + StringUtils.format("Timestamp[%s] > expectedLast[%s]", current, expectedLast), + descending ? current.isBefore(expectedLast) : current.isAfter(expectedLast) + ); + + final TimeseriesResultValue value = result.getValue(); + + Assert.assertEquals( + value.getLongMetric(TIMESTAMP_RESULT_FIELD_NAME), + current.getMillis(), + 0 + ); + + Assert.assertEquals( + result.toString(), + QueryRunnerTestHelper.SKIPPED_DAY.equals(current) ? 0L : 13L, + value.getLongMetric("rows").longValue() + ); + + if (!QueryRunnerTestHelper.SKIPPED_DAY.equals(current)) { + Assert.assertEquals( + result.toString(), + Doubles.tryParse(expectedIndex[count]).doubleValue(), + value.getDoubleMetric("index").doubleValue(), + value.getDoubleMetric("index").doubleValue() * 1e-6 + ); + Assert.assertEquals( + result.toString(), + new Double(expectedIndex[count]) + + 13L + 1L, + value.getDoubleMetric("addRowsIndexConstant"), + value.getDoubleMetric("addRowsIndexConstant") * 1e-6 + ); + Assert.assertEquals( + value.getDoubleMetric("uniques"), + 9.0d, + 0.02 + ); + } else { + if (NullHandling.replaceWithDefault()) { + Assert.assertEquals( + result.toString(), + 0.0D, + value.getDoubleMetric("index").doubleValue(), + value.getDoubleMetric("index").doubleValue() * 1e-6 + ); + Assert.assertEquals( + result.toString(), + new Double(expectedIndex[count]) + 1L, + value.getDoubleMetric("addRowsIndexConstant"), + value.getDoubleMetric("addRowsIndexConstant") * 1e-6 + ); + Assert.assertEquals( + 0.0D, + value.getDoubleMetric("uniques"), + 0.02 + ); + } else { + Assert.assertNull( + result.toString(), + value.getDoubleMetric("index") + ); + Assert.assertNull( + result.toString(), + value.getDoubleMetric("addRowsIndexConstant") + ); + Assert.assertEquals( + value.getDoubleMetric("uniques"), + 0.0d, + 0.02 + ); + } + } + + lastResult = result; + ++count; + } + + Assert.assertEquals(lastResult.toString(), expectedLast, lastResult.getTimestamp()); + } + @Test public void testTimeSeriesWithSelectionFilterLookupExtractionFn() { 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 48fd49c0d17d..f98160a1a390 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 @@ -175,6 +175,15 @@ public static List>> queryRunners() ) ) ); + + RESOURCE_CLOSER.register(() -> { + // Verify that all objects have been returned to the pool. + Assert.assertEquals("defaultPool objects created", defaultPool.poolSize(), defaultPool.objectsCreatedCount()); + Assert.assertEquals("customPool objects created", customPool.poolSize(), customPool.objectsCreatedCount()); + defaultPool.close(); + customPool.close(); + }); + return retVal; } @@ -638,15 +647,15 @@ public void testTopNOverMissingUniques() Arrays.>asList( ImmutableMap.builder() .put("market", "spot") - .put("uniques", 0) + .put("uniques", 0d) .build(), ImmutableMap.builder() .put("market", "total_market") - .put("uniques", 0) + .put("uniques", 0d) .build(), ImmutableMap.builder() .put("market", "upfront") - .put("uniques", 0) + .put("uniques", 0d) .build() ) ) diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java index 7a2607662380..f5797dba3c9f 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNUnionQueryTest.java @@ -33,6 +33,7 @@ import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory; import org.apache.druid.segment.TestHelper; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.AfterClass; import org.junit.Test; import org.junit.runner.RunWith; @@ -46,7 +47,7 @@ import java.util.Map; @RunWith(Parameterized.class) -public class TopNUnionQueryTest +public class TopNUnionQueryTest extends InitializedNullHandlingTest { private static final Closer RESOURCE_CLOSER = Closer.create(); diff --git a/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.java new file mode 100644 index 000000000000..8b4a8d3f5313 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/CustomSegmentizerFactoryTest.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; + +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.jackson.SegmentizerModule; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory; +import org.apache.druid.segment.loading.SegmentLoadingException; +import org.apache.druid.segment.loading.SegmentizerFactory; +import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.apache.druid.timeline.DataSegment; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.IOException; + +public class CustomSegmentizerFactoryTest extends InitializedNullHandlingTest +{ + private static ObjectMapper JSON_MAPPER; + private static IndexIO INDEX_IO; + private static IndexMerger INDEX_MERGER; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + @BeforeClass + public static void setup() + { + final ObjectMapper mapper = new DefaultObjectMapper(); + mapper.registerModule(new SegmentizerModule()); + mapper.registerSubtypes(new NamedType(CustomSegmentizerFactory.class, "customSegmentFactory")); + final IndexIO indexIO = new IndexIO(mapper, () -> 0); + + mapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ExprMacroTable.class.getName(), TestExprMacroTable.INSTANCE) + .addValue(ObjectMapper.class.getName(), mapper) + .addValue(IndexIO.class, indexIO) + .addValue(DataSegment.PruneSpecsHolder.class, DataSegment.PruneSpecsHolder.DEFAULT) + ); + + JSON_MAPPER = mapper; + INDEX_IO = indexIO; + INDEX_MERGER = new IndexMergerV9(mapper, indexIO, OffHeapMemorySegmentWriteOutMediumFactory.instance()); + } + + @Test + public void testDefaultSegmentizerPersist() throws IOException + { + IncrementalIndex data = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv"); + File segment = new File(temporaryFolder.newFolder(), "segment"); + File persisted = INDEX_MERGER.persist( + data, + Intervals.of("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"), + segment, + new IndexSpec( + null, + null, + null, + null, + null + ), + null + ); + + File factoryJson = new File(persisted, "factory.json"); + Assert.assertTrue(factoryJson.exists()); + SegmentizerFactory factory = JSON_MAPPER.readValue(factoryJson, SegmentizerFactory.class); + Assert.assertTrue(factory instanceof MMappedQueryableSegmentizerFactory); + } + + @Test + public void testCustomSegmentizerPersist() throws IOException + { + IncrementalIndex data = TestIndex.makeRealtimeIndex("druid.sample.numeric.tsv"); + File segment = new File(temporaryFolder.newFolder(), "segment"); + File persisted = INDEX_MERGER.persist( + data, + Intervals.of("2011-01-12T00:00:00.000Z/2011-05-01T00:00:00.000Z"), + segment, + new IndexSpec( + null, + null, + null, + null, + new CustomSegmentizerFactory() + ), + null + ); + + File factoryJson = new File(persisted, "factory.json"); + Assert.assertTrue(factoryJson.exists()); + SegmentizerFactory factory = JSON_MAPPER.readValue(factoryJson, SegmentizerFactory.class); + Assert.assertTrue(factory instanceof CustomSegmentizerFactory); + } + + private static class CustomSegmentizerFactory implements SegmentizerFactory + { + @Override + public Segment factorize(DataSegment segment, File parentDir, boolean lazy) throws SegmentLoadingException + { + try { + return new QueryableIndexSegment(INDEX_IO.loadIndex(parentDir, lazy), segment.getId()); + } + catch (IOException e) { + throw new SegmentLoadingException(e, "%s", e.getMessage()); + } + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java index 1c68a21568d3..b4525c018eee 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerV9WithSpatialIndexTest.java @@ -48,6 +48,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; import org.junit.Test; import org.junit.runner.RunWith; @@ -66,7 +67,7 @@ /** */ @RunWith(Parameterized.class) -public class IndexMergerV9WithSpatialIndexTest +public class IndexMergerV9WithSpatialIndexTest extends InitializedNullHandlingTest { public static final int NUM_POINTS = 5000; diff --git a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java index 4e03668f68ba..e9197ac8647a 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexSpecTest.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; @@ -65,4 +66,10 @@ public void testDefaults() Assert.assertEquals(CompressionStrategy.LZ4, spec.getMetricCompression()); Assert.assertEquals(CompressionFactory.LongEncodingStrategy.LONGS, spec.getLongEncoding()); } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(IndexSpec.class).usingGetClass().verify(); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java b/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java index dc102e688164..c7783e992197 100644 --- a/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/QueryableIndexColumnCapabilitiesTest.java @@ -38,6 +38,7 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory; 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.ValueType; import org.apache.druid.segment.incremental.IncrementalIndex; @@ -152,9 +153,12 @@ public void testStringColumn() Assert.assertTrue(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + // multi-value is unknown unless explicitly set to 'true' + Assert.assertTrue(caps.hasMultipleValues().isUnknown()); + // at index merge or query time we 'complete' the capabilities to take a snapshot of the current state, + // coercing any 'UNKNOWN' values to false + Assert.assertFalse(ColumnCapabilitiesImpl.snapshot(caps).hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); caps = MMAP_INDEX.getColumnHolder("d1").getCapabilities(); Assert.assertEquals(ValueType.STRING, caps.getType()); @@ -162,9 +166,8 @@ public void testStringColumn() Assert.assertTrue(caps.isDictionaryEncoded()); Assert.assertTrue(caps.areDictionaryValuesSorted().isTrue()); Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); } @Test @@ -176,9 +179,8 @@ public void testMultiStringColumn() Assert.assertTrue(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertTrue(caps.hasMultipleValues()); + Assert.assertTrue(caps.hasMultipleValues().isTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); caps = MMAP_INDEX.getColumnHolder("d2").getCapabilities(); Assert.assertEquals(ValueType.STRING, caps.getType()); @@ -186,9 +188,8 @@ public void testMultiStringColumn() Assert.assertTrue(caps.isDictionaryEncoded()); Assert.assertTrue(caps.areDictionaryValuesSorted().isTrue()); Assert.assertTrue(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertTrue(caps.hasMultipleValues()); + Assert.assertTrue(caps.hasMultipleValues().isTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); } @Test @@ -206,8 +207,7 @@ private void assertNonStringColumnCapabilities(ColumnCapabilities caps, ValueTyp Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); } } 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 cee0a46e2896..6566592e9d5c 100644 --- a/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ReferenceCountingSegmentTest.java @@ -21,6 +21,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.timeline.SegmentId; +import org.easymock.EasyMock; import org.joda.time.Days; import org.joda.time.Interval; import org.junit.Assert; @@ -38,39 +39,50 @@ public class ReferenceCountingSegmentTest private ReferenceCountingSegment segment; private ExecutorService exec; + 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 int underlyingSegmentClosedCount; + @Before public void setUp() { + underlyingSegmentClosedCount = 0; + index = EasyMock.createNiceMock(QueryableIndex.class); + adapter = EasyMock.createNiceMock(StorageAdapter.class); + segment = ReferenceCountingSegment.wrapRootGenerationSegment( - new AbstractSegment() + new Segment() { @Override public SegmentId getId() { - return SegmentId.dummy("test_segment"); + return segmentId; } @Override public Interval getDataInterval() { - return new Interval(DateTimes.nowUtc().minus(Days.days(1)), DateTimes.nowUtc()); + return dataInterval; } @Override public QueryableIndex asQueryableIndex() { - return null; + return index; } @Override public StorageAdapter asStorageAdapter() { - return null; + return adapter; } @Override public void close() { + underlyingSegmentClosedCount++; } } ); @@ -81,13 +93,17 @@ public void close() @Test public void testMultipleClose() throws Exception { + Assert.assertEquals(0, underlyingSegmentClosedCount); Assert.assertFalse(segment.isClosed()); Assert.assertTrue(segment.increment()); Assert.assertEquals(1, segment.getNumReferences()); Closeable closeable = segment.decrementOnceCloseable(); + Assert.assertEquals(0, underlyingSegmentClosedCount); closeable.close(); + Assert.assertEquals(0, underlyingSegmentClosedCount); closeable.close(); + Assert.assertEquals(0, underlyingSegmentClosedCount); exec.submit( () -> { try { @@ -99,10 +115,16 @@ public void testMultipleClose() throws Exception } ).get(); Assert.assertEquals(0, segment.getNumReferences()); + Assert.assertEquals(0, underlyingSegmentClosedCount); Assert.assertFalse(segment.isClosed()); + // close for reals segment.close(); + Assert.assertTrue(segment.isClosed()); + Assert.assertEquals(1, underlyingSegmentClosedCount); + // ... but make sure it only happens once segment.close(); + Assert.assertEquals(1, underlyingSegmentClosedCount); exec.submit( () -> { try { @@ -116,13 +138,25 @@ public void testMultipleClose() throws Exception Assert.assertEquals(0, segment.getNumReferences()); Assert.assertTrue(segment.isClosed()); + Assert.assertEquals(1, underlyingSegmentClosedCount); segment.increment(); segment.increment(); segment.increment(); Assert.assertEquals(0, segment.getNumReferences()); - + Assert.assertEquals(1, underlyingSegmentClosedCount); segment.close(); Assert.assertEquals(0, segment.getNumReferences()); + Assert.assertEquals(1, underlyingSegmentClosedCount); } + + @Test + public void testExposesWrappedSegment() + { + Assert.assertEquals(segmentId, segment.getId()); + Assert.assertEquals(dataInterval, segment.getDataInterval()); + Assert.assertEquals(index, segment.asQueryableIndex()); + Assert.assertEquals(adapter, segment.asStorageAdapter()); + } + } diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java index 8886c7d2d8f1..e12dac4743cd 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedColumnSelectorFactoryTest.java @@ -54,9 +54,8 @@ public void testCapabilitiesTime() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); } @Test @@ -69,9 +68,8 @@ public void testCapabilitiesString() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertTrue(caps.hasMultipleValues().isUnknown()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertFalse(caps.isComplete()); } @Test @@ -84,9 +82,8 @@ public void testCapabilitiesLong() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); } @Test @@ -99,9 +96,8 @@ public void testCapabilitiesFloat() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); } @Test @@ -114,9 +110,8 @@ public void testCapabilitiesDouble() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertFalse(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertTrue(caps.isComplete()); } @Test @@ -129,9 +124,8 @@ public void testCapabilitiesComplex() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertFalse(caps.hasMultipleValues()); + Assert.assertTrue(caps.hasMultipleValues().isUnknown()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertFalse(caps.isComplete()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java index a4fc2f254b78..652a59033fab 100644 --- a/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/RowBasedStorageAdapterTest.java @@ -332,8 +332,7 @@ public void test_getColumnCapabilities_float() final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.FLOAT.name()); Assert.assertEquals(ValueType.FLOAT, capabilities.getType()); - Assert.assertFalse(capabilities.hasMultipleValues()); - Assert.assertTrue(capabilities.isComplete()); + Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue()); } @Test @@ -343,8 +342,7 @@ public void test_getColumnCapabilities_double() final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.DOUBLE.name()); Assert.assertEquals(ValueType.DOUBLE, capabilities.getType()); - Assert.assertFalse(capabilities.hasMultipleValues()); - Assert.assertTrue(capabilities.isComplete()); + Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue()); } @Test @@ -354,8 +352,7 @@ public void test_getColumnCapabilities_long() final ColumnCapabilities capabilities = adapter.getColumnCapabilities(ValueType.LONG.name()); Assert.assertEquals(ValueType.LONG, capabilities.getType()); - Assert.assertFalse(capabilities.hasMultipleValues()); - Assert.assertTrue(capabilities.isComplete()); + Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue()); } @Test @@ -369,8 +366,7 @@ public void test_getColumnCapabilities_string() // Note: unlike numeric types, STRING-typed columns might have multiple values, so they report as incomplete. It // would be good in the future to support some way of changing this, when it is known ahead of time that // multi-valuedness is definitely happening or is definitely impossible. - Assert.assertFalse(capabilities.hasMultipleValues()); - Assert.assertFalse(capabilities.isComplete()); + Assert.assertTrue(capabilities.hasMultipleValues().isUnknown()); } @Test @@ -382,8 +378,7 @@ public void test_getColumnCapabilities_complex() // Note: unlike numeric types, COMPLEX-typed columns report that they are incomplete. Assert.assertEquals(ValueType.COMPLEX, capabilities.getType()); - Assert.assertFalse(capabilities.hasMultipleValues()); - Assert.assertFalse(capabilities.isComplete()); + Assert.assertTrue(capabilities.hasMultipleValues().isUnknown()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java b/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java index 53e93c7a3b54..e221edd9c73f 100644 --- a/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java +++ b/processing/src/test/java/org/apache/druid/segment/column/ColumnCapabilitiesImplTest.java @@ -47,7 +47,7 @@ public void testSerde() throws Exception Assert.assertTrue(cc.isDictionaryEncoded()); Assert.assertFalse(cc.isRunLengthEncoded()); Assert.assertTrue(cc.hasSpatialIndexes()); - Assert.assertTrue(cc.hasMultipleValues()); + Assert.assertTrue(cc.hasMultipleValues().isTrue()); Assert.assertTrue(cc.hasBitmapIndexes()); Assert.assertFalse(cc.isFilterable()); } @@ -72,7 +72,7 @@ public void testDeserialization() throws Exception Assert.assertTrue(cc.isDictionaryEncoded()); Assert.assertTrue(cc.isRunLengthEncoded()); Assert.assertTrue(cc.hasSpatialIndexes()); - Assert.assertTrue(cc.hasMultipleValues()); + Assert.assertTrue(cc.hasMultipleValues().isTrue()); Assert.assertTrue(cc.hasBitmapIndexes()); Assert.assertFalse(cc.isFilterable()); } diff --git a/processing/src/test/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplierTest.java index 3b3d427a23ae..f113dc3510ad 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/CompressedVSizeColumnarIntsSupplierTest.java @@ -56,13 +56,8 @@ public class CompressedVSizeColumnarIntsSupplierTest extends CompressionStrategy @Parameterized.Parameters(name = "{index}: compression={0}, byteOrder={1}") public static Iterable compressionStrategies() { - final Iterable compressionStrategies = Iterables.transform( - CompressionStrategyTest.compressionStrategies(), - (Object[] input) -> (CompressionStrategy) input[0] - ); - Set> combinations = Sets.cartesianProduct( - Sets.newHashSet(compressionStrategies), + Sets.newHashSet(CompressionStrategy.noNoneValues()), Sets.newHashSet(ByteOrder.BIG_ENDIAN, ByteOrder.LITTLE_ENDIAN) ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java index c7a560c19616..3ed33b8e3fc6 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExtractionDimFilterTest.java @@ -36,6 +36,7 @@ import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.BitmapIndex; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; @@ -146,9 +147,9 @@ public Iterator iterator() } @Override - public boolean hasMultipleValues(final String dimension) + public ColumnCapabilities.Capable hasMultipleValues(final String dimension) { - return true; + return ColumnCapabilities.Capable.TRUE; } @Override diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java index a1692854328e..c12c012b5e05 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.filter; 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.StringUtils; import org.apache.druid.query.dimension.DimensionSpec; @@ -35,6 +36,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -155,6 +157,51 @@ public void testToCnfWithMuchReducibleFilter() assertFilter(muchReducible, expected, cnf); } + @Test + public void testToNormalizedOrClausesWithMuchReducibleFilter() + { + final Filter muchReducible = FilterTestUtils.and( + // should be flattened + FilterTestUtils.and( + FilterTestUtils.and( + FilterTestUtils.and(FilterTestUtils.selector("col1", "val1")) + ) + ), + // should be flattened + FilterTestUtils.and( + FilterTestUtils.or( + FilterTestUtils.and(FilterTestUtils.selector("col1", "val1")) + ) + ), + // should be flattened + FilterTestUtils.or( + FilterTestUtils.and( + FilterTestUtils.or(FilterTestUtils.selector("col1", "val1")) + ) + ), + // should eliminate duplicate filters + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val2"), + FilterTestUtils.and( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val2") + ), + FilterTestUtils.and( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.and( + FilterTestUtils.selector("col2", "val2"), + FilterTestUtils.selector("col1", "val1") + ) + ) + ); + final Set expected = ImmutableSet.of( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val2") + ); + final Set normalizedOrClauses = Filters.toNormalizedOrClauses(muchReducible); + Assert.assertEquals(expected, normalizedOrClauses); + } + @Test public void testToCnfWithComplexFilterIncludingNotAndOr() { @@ -259,6 +306,110 @@ public void testToCnfWithComplexFilterIncludingNotAndOr() assertFilter(filter, expected, cnf); } + @Test + public void testToNormalizedOrClausesWithComplexFilterIncludingNotAndOr() + { + final Filter filter = FilterTestUtils.and( + FilterTestUtils.or( + FilterTestUtils.and( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val2") + ), + FilterTestUtils.not( + FilterTestUtils.and( + FilterTestUtils.selector("col4", "val4"), + FilterTestUtils.selector("col5", "val5") + ) + ) + ), + FilterTestUtils.or( + FilterTestUtils.not( + FilterTestUtils.or( + FilterTestUtils.selector("col2", "val2"), + FilterTestUtils.selector("col4", "val4"), + FilterTestUtils.selector("col5", "val5") + ) + ), + FilterTestUtils.and( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col3", "val3") + ) + ), + FilterTestUtils.and( + FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val22"), // selecting different value + FilterTestUtils.selector("col3", "val3") + ), + FilterTestUtils.not( + FilterTestUtils.selector("col1", "val11") + ) + ), + FilterTestUtils.and( + FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val22"), + FilterTestUtils.selector("col3", "val3") + ), + FilterTestUtils.not( + FilterTestUtils.selector("col1", "val11") // selecting different value + ) + ) + ); + final Set expected = ImmutableSet.of( + FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val22"), + FilterTestUtils.selector("col3", "val3") + ), + FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.not(FilterTestUtils.selector("col2", "val2")) + ), + FilterTestUtils.or( + FilterTestUtils.not(FilterTestUtils.selector("col2", "val2")), + FilterTestUtils.selector("col3", "val3") + ), + FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.not(FilterTestUtils.selector("col4", "val4")) + ), + FilterTestUtils.or( + FilterTestUtils.selector("col3", "val3"), + FilterTestUtils.not(FilterTestUtils.selector("col4", "val4")) + ), + FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.not(FilterTestUtils.selector("col5", "val5")) + ), + FilterTestUtils.or( + FilterTestUtils.selector("col3", "val3"), + FilterTestUtils.not(FilterTestUtils.selector("col5", "val5")) + ), + FilterTestUtils.not(FilterTestUtils.selector("col1", "val11")), + // The below OR filter could be eliminated because this filter also has + // (col1 = val1 || ~(col4 = val4)) && (col1 = val1 || ~(col5 = val5)). + // The reduction process would be + // (col1 = val1 || ~(col4 = val4)) && (col1 = val1 || ~(col5 = val5)) && (col1 = val1 || ~(col4 = val4) || ~(col5 = val5)) + // => (col1 = val1 && ~(col4 = val4) || ~(col5 = val5)) && (col1 = val1 || ~(col4 = val4) || ~(col5 = val5)) + // => (col1 = val1 && ~(col4 = val4) || ~(col5 = val5)) + // => (col1 = val1 || ~(col4 = val4)) && (col1 = val1 || ~(col5 = val5)). + // However, we don't have this reduction now, so we have a filter in a suboptimized CNF. + FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.not(FilterTestUtils.selector("col4", "val4")), + FilterTestUtils.not(FilterTestUtils.selector("col5", "val5")) + ), + FilterTestUtils.or( + FilterTestUtils.selector("col2", "val2"), + FilterTestUtils.not(FilterTestUtils.selector("col4", "val4")), + FilterTestUtils.not(FilterTestUtils.selector("col5", "val5")) + ) + ); + final Set normalizedOrClauses = Filters.toNormalizedOrClauses(filter); + Assert.assertEquals(expected, normalizedOrClauses); + } + @Test public void testToCnfCollapsibleBigFilter() { @@ -355,6 +506,18 @@ public void testToCnfFilterThatPullCannotConvertToCnfProperly() assertFilter(filter, expectedCnf, Filters.toCnf(filter)); } + @Test + public void testToNormalizedOrClausesNonAndFilterShouldReturnSingleton() + { + Filter filter = FilterTestUtils.or( + FilterTestUtils.selector("col1", "val1"), + FilterTestUtils.selector("col2", "val2") + ); + Set expected = Collections.singleton(filter); + Set normalizedOrClauses = Filters.toNormalizedOrClauses(filter); + Assert.assertEquals(expected, normalizedOrClauses); + } + @Test public void testTrueFalseFilterRequiredColumnRewrite() { diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java new file mode 100644 index 000000000000..6698c3ab6ac1 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java @@ -0,0 +1,152 @@ +/* + * 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.filter; + +import com.google.common.collect.ImmutableList; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.SimpleAscendingOffset; +import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.VSizeColumnarInts; +import org.apache.druid.segment.data.VSizeColumnarMultiInts; +import org.apache.druid.segment.serde.DictionaryEncodedColumnSupplier; +import org.apache.druid.segment.vector.NilVectorSelector; +import org.apache.druid.segment.vector.NoFilterVectorOffset; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class ValueMatchersTest extends InitializedNullHandlingTest +{ + private DictionaryEncodedColumnSupplier supplierSingleConstant; + private DictionaryEncodedColumnSupplier supplierSingle; + private DictionaryEncodedColumnSupplier supplierMulti; + + @Before + public void setup() + { + supplierSingleConstant = new DictionaryEncodedColumnSupplier( + GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY), + () -> VSizeColumnarInts.fromArray(new int[]{0}), + null, + 0 + ); + supplierSingle = new DictionaryEncodedColumnSupplier( + GenericIndexed.fromIterable(ImmutableList.of("value", "value2"), GenericIndexed.STRING_STRATEGY), + () -> VSizeColumnarInts.fromArray(new int[]{0, 0, 1, 0, 1}), + null, + 0 + ); + supplierMulti = new DictionaryEncodedColumnSupplier( + GenericIndexed.fromIterable(ImmutableList.of("value"), GenericIndexed.STRING_STRATEGY), + null, + () -> VSizeColumnarMultiInts.fromIterable( + ImmutableList.of( + VSizeColumnarInts.fromArray(new int[]{0, 0}), + VSizeColumnarInts.fromArray(new int[]{0}) + ) + ), + 0 + ); + } + @Test + public void testNullDimensionSelectorCanBeBoolean() + { + Boolean resultMatchNull = ValueMatchers.toBooleanIfPossible( + DimensionSelector.constant(null), + false, + string -> string == null + ); + Assert.assertNotNull(resultMatchNull); + Assert.assertTrue(resultMatchNull); + + Boolean resultMatchNotNull = ValueMatchers.toBooleanIfPossible( + DimensionSelector.constant(null), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNotNull); + Assert.assertFalse(resultMatchNotNull); + + Boolean resultMatchNonNilConstant = ValueMatchers.toBooleanIfPossible( + supplierSingleConstant.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNonNilConstant); + Assert.assertTrue(resultMatchNonNilConstant); + + Boolean resultMatchNonNil = ValueMatchers.toBooleanIfPossible( + supplierSingle.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), + false, + string -> string != null + ); + Assert.assertNull(resultMatchNonNil); + + Boolean resultMatchNonNilMulti = ValueMatchers.toBooleanIfPossible( + supplierMulti.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), + true, + string -> string != null + ); + Assert.assertNull(resultMatchNonNilMulti); + } + + @Test + public void testNilVectorSelectorCanBeBoolean() + { + Boolean resultMatchNull = ValueMatchers.toBooleanIfPossible( + NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)), + false, + string -> string == null + ); + Assert.assertNotNull(resultMatchNull); + Assert.assertTrue(resultMatchNull); + + Boolean resultMatchNotNull = ValueMatchers.toBooleanIfPossible( + NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNotNull); + Assert.assertFalse(resultMatchNotNull); + + Boolean resultMatchNotNilConstant = ValueMatchers.toBooleanIfPossible( + supplierSingleConstant.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), + false, + string -> string != null + ); + Assert.assertNotNull(resultMatchNotNilConstant); + Assert.assertTrue(resultMatchNotNilConstant); + + Boolean resultMatchNotNil = ValueMatchers.toBooleanIfPossible( + supplierSingle.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), + false, + string -> string != null + ); + Assert.assertNull(resultMatchNotNil); + + Boolean resultMatchNotNilMulti = ValueMatchers.toBooleanIfPossible( + supplierMulti.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), + true, + string -> string != null + ); + Assert.assertNull(resultMatchNotNilMulti); + } +} diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/BenchmarkDataGeneratorTest.java b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java similarity index 72% rename from benchmarks/src/test/java/org/apache/druid/benchmark/BenchmarkDataGeneratorTest.java rename to processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java index 035e33079ce9..4f65d377d4ee 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/BenchmarkDataGeneratorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/DataGeneratorTest.java @@ -17,13 +17,13 @@ * under the License. */ -package org.apache.druid.benchmark; +package org.apache.druid.segment.generator; -import org.apache.druid.benchmark.datagen.BenchmarkColumnSchema; -import org.apache.druid.benchmark.datagen.BenchmarkDataGenerator; +import org.apache.commons.math3.distribution.NormalDistribution; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.column.ValueType; +import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; @@ -34,16 +34,16 @@ import java.util.Map; // Doesn't assert behavior right now, just generates rows and prints out some distribution numbers -public class BenchmarkDataGeneratorTest +public class DataGeneratorTest { @Test public void testSequential() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); RowValueTracker tracker = new RowValueTracker(); schemas.add( - BenchmarkColumnSchema.makeSequential( + GeneratorColumnSchema.makeSequential( "dimA", ValueType.STRING, false, @@ -55,7 +55,7 @@ public void testSequential() ); schemas.add( - BenchmarkColumnSchema.makeEnumeratedSequential( + GeneratorColumnSchema.makeEnumeratedSequential( "dimB", ValueType.STRING, false, @@ -66,7 +66,7 @@ public void testSequential() ); schemas.add( - BenchmarkColumnSchema.makeSequential( + GeneratorColumnSchema.makeSequential( "dimC", ValueType.STRING, false, @@ -77,7 +77,7 @@ public void testSequential() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, 0, 0, 1000.0); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0); for (int i = 0; i < 100; i++) { InputRow row = dataGenerator.nextRow(); //System.out.println("S-ROW: " + row); @@ -89,11 +89,11 @@ public void testSequential() @Test public void testDiscreteUniform() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); RowValueTracker tracker = new RowValueTracker(); schemas.add( - BenchmarkColumnSchema.makeDiscreteUniform( + GeneratorColumnSchema.makeDiscreteUniform( "dimA", ValueType.STRING, false, @@ -105,7 +105,7 @@ public void testDiscreteUniform() ); schemas.add( - BenchmarkColumnSchema.makeEnumeratedDiscreteUniform( + GeneratorColumnSchema.makeEnumeratedDiscreteUniform( "dimB", ValueType.STRING, false, @@ -116,7 +116,7 @@ public void testDiscreteUniform() ); schemas.add( - BenchmarkColumnSchema.makeDiscreteUniform( + GeneratorColumnSchema.makeDiscreteUniform( "dimC", ValueType.STRING, false, @@ -128,7 +128,7 @@ public void testDiscreteUniform() ); schemas.add( - BenchmarkColumnSchema.makeDiscreteUniform( + GeneratorColumnSchema.makeDiscreteUniform( "dimD", ValueType.FLOAT, false, @@ -139,7 +139,7 @@ public void testDiscreteUniform() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, 0, 0, 1000.0); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0); for (int i = 0; i < 100; i++) { InputRow row = dataGenerator.nextRow(); //System.out.println("U-ROW: " + row); @@ -154,11 +154,11 @@ public void testDiscreteUniform() @Test public void testRoundedNormal() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); RowValueTracker tracker = new RowValueTracker(); schemas.add( - BenchmarkColumnSchema.makeNormal( + GeneratorColumnSchema.makeNormal( "dimA", ValueType.FLOAT, false, @@ -171,7 +171,7 @@ public void testRoundedNormal() ); schemas.add( - BenchmarkColumnSchema.makeNormal( + GeneratorColumnSchema.makeNormal( "dimB", ValueType.STRING, false, @@ -183,7 +183,7 @@ public void testRoundedNormal() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, 0, 0, 1000.0); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0); for (int i = 0; i < 1000000; i++) { InputRow row = dataGenerator.nextRow(); //System.out.println("N-ROW: " + row); @@ -197,11 +197,11 @@ public void testRoundedNormal() @Test public void testZipf() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); RowValueTracker tracker = new RowValueTracker(); schemas.add( - BenchmarkColumnSchema.makeZipf( + GeneratorColumnSchema.makeZipf( "dimA", ValueType.STRING, false, @@ -214,7 +214,7 @@ public void testZipf() ); schemas.add( - BenchmarkColumnSchema.makeZipf( + GeneratorColumnSchema.makeZipf( "dimB", ValueType.FLOAT, false, @@ -227,7 +227,7 @@ public void testZipf() ); schemas.add( - BenchmarkColumnSchema.makeEnumeratedZipf( + GeneratorColumnSchema.makeEnumeratedZipf( "dimC", ValueType.STRING, false, @@ -238,7 +238,7 @@ public void testZipf() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, 0, 0, 1000.0); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0); for (int i = 0; i < 100; i++) { InputRow row = dataGenerator.nextRow(); //System.out.println("Z-ROW: " + row); @@ -252,11 +252,11 @@ public void testZipf() @Test public void testEnumerated() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); RowValueTracker tracker = new RowValueTracker(); schemas.add( - BenchmarkColumnSchema.makeEnumerated( + GeneratorColumnSchema.makeEnumerated( "dimA", ValueType.STRING, false, @@ -267,7 +267,7 @@ public void testEnumerated() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, 0, 0, 1000.0); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0); for (int i = 0; i < 10000; i++) { InputRow row = dataGenerator.nextRow(); //System.out.println("Z-ROW: " + row); @@ -281,11 +281,11 @@ public void testEnumerated() @Test public void testNormal() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); RowValueTracker tracker = new RowValueTracker(); schemas.add( - BenchmarkColumnSchema.makeNormal( + GeneratorColumnSchema.makeNormal( "dimA", ValueType.FLOAT, false, @@ -298,7 +298,7 @@ public void testNormal() ); schemas.add( - BenchmarkColumnSchema.makeNormal( + GeneratorColumnSchema.makeNormal( "dimB", ValueType.STRING, false, @@ -310,7 +310,7 @@ public void testNormal() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, 0, 0, 1000.0); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0); for (int i = 0; i < 100; i++) { InputRow row = dataGenerator.nextRow(); //System.out.println("N-ROW: " + row); @@ -324,11 +324,11 @@ public void testNormal() @Test public void testRealUniform() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); RowValueTracker tracker = new RowValueTracker(); schemas.add( - BenchmarkColumnSchema.makeContinuousUniform( + GeneratorColumnSchema.makeContinuousUniform( "dimA", ValueType.STRING, false, @@ -340,7 +340,7 @@ public void testRealUniform() ); schemas.add( - BenchmarkColumnSchema.makeContinuousUniform( + GeneratorColumnSchema.makeContinuousUniform( "dimB", ValueType.FLOAT, false, @@ -351,7 +351,7 @@ public void testRealUniform() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, 0, 0, 1000.0); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, 0, 0, 1000.0); for (int i = 0; i < 100; i++) { InputRow row = dataGenerator.nextRow(); //System.out.println("U-ROW: " + row); @@ -365,10 +365,10 @@ public void testRealUniform() @Test public void testIntervalBasedTimeGeneration() { - List schemas = new ArrayList<>(); + List schemas = new ArrayList<>(); schemas.add( - BenchmarkColumnSchema.makeEnumeratedSequential( + GeneratorColumnSchema.makeEnumeratedSequential( "dimB", ValueType.STRING, false, @@ -378,17 +378,38 @@ public void testIntervalBasedTimeGeneration() ) ); - BenchmarkDataGenerator dataGenerator = new BenchmarkDataGenerator(schemas, 9999, Intervals.utc(50000, 600000), 100); + DataGenerator dataGenerator = new DataGenerator(schemas, 9999, Intervals.utc(50000, 600000), 100); for (int i = 0; i < 100; i++) { dataGenerator.nextRow(); } - BenchmarkDataGenerator dataGenerator2 = new BenchmarkDataGenerator(schemas, 9999, Intervals.utc(50000, 50001), 100); + DataGenerator dataGenerator2 = new DataGenerator(schemas, 9999, Intervals.utc(50000, 50001), 100); for (int i = 0; i < 100; i++) { dataGenerator2.nextRow(); } } + @Test + public void testBasicSchemasAndGeneratorSchemaInfo() + { + GeneratorSchemaInfo basicSchema = GeneratorBasicSchemas.SCHEMA_MAP.get("basic"); + Assert.assertEquals(13, basicSchema.getColumnSchemas().size()); + Assert.assertEquals(6, basicSchema.getAggs().size()); + Assert.assertEquals(6, basicSchema.getAggsArray().length); + Assert.assertNotNull(basicSchema.getDimensionsSpec()); + Assert.assertNotNull(basicSchema.getDataInterval()); + Assert.assertTrue(basicSchema.isWithRollup()); + } + + @Test + public void testRealRoundingDistributionZeroGetters() + { + RealRoundingDistribution dist = new RealRoundingDistribution(new NormalDistribution()); + Assert.assertEquals(0, dist.getSupportLowerBound()); + Assert.assertEquals(0, dist.getSupportUpperBound()); + Assert.assertEquals(0, dist.getNumericalMean(), 0); + Assert.assertEquals(0, dist.getNumericalVariance(), 0); + } private static class RowValueTracker { diff --git a/processing/src/test/java/org/apache/druid/segment/generator/GeneratorColumnSchemaTest.java b/processing/src/test/java/org/apache/druid/segment/generator/GeneratorColumnSchemaTest.java new file mode 100644 index 000000000000..4edf6bd362c4 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/generator/GeneratorColumnSchemaTest.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.segment.generator; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import nl.jqno.equalsverifier.EqualsVerifier; +import nl.jqno.equalsverifier.Warning; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.segment.column.ValueType; +import org.junit.Assert; +import org.junit.Test; + +public class GeneratorColumnSchemaTest +{ + private static final ObjectMapper MAPPER = new DefaultObjectMapper(); + + @Test + public void testSerde() throws JsonProcessingException + { + GeneratorColumnSchema schema = GeneratorColumnSchema.makeLazyZipf( + "test", + ValueType.LONG, + false, + 1, + 0.0, + 0, + 1000, + 1.3 + ); + + final String serialized = MAPPER.writeValueAsString(schema); + GeneratorColumnSchema again = MAPPER.readValue(serialized, GeneratorColumnSchema.class); + Assert.assertEquals(schema, again); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(GeneratorColumnSchema.class) + .withNonnullFields("name", "distributionType", "type") + .suppress(Warning.NONFINAL_FIELDS) + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java index a01a34b9c23c..5fd495340b82 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/BaseHashJoinSegmentStorageAdapterTest.java @@ -24,11 +24,12 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.filter.Filter; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTable; import org.apache.druid.segment.join.table.IndexedTableJoinable; @@ -43,9 +44,17 @@ import org.junit.rules.TemporaryFolder; import java.io.IOException; +import java.util.List; public class BaseHashJoinSegmentStorageAdapterTest { + public static JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ); + public static final String FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX = "c1."; public static final String FACT_TO_COUNTRY_ON_NUMBER_PREFIX = "c2."; public static final String FACT_TO_REGION_PREFIX = "r1."; @@ -186,20 +195,12 @@ protected JoinableClause regionToCountry(final JoinType joinType) protected HashJoinSegmentStorageAdapter makeFactToCountrySegment() { - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)), - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup(); return new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)), - preAnalysis + joinFilterPreAnalysisGroup ); } @@ -221,4 +222,32 @@ protected void compareExpressionVirtualColumns( actualVirtualColumn.getParsedExpression().get().toString() ); } + + protected static JoinFilterPreAnalysisGroup makeDefaultConfigPreAnalysisGroup() + { + return new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + true + ); + } + + protected static JoinFilterPreAnalysisGroup makeDefaultConfigPreAnalysisGroup( + Filter originalFilter, + List joinableClauses, + VirtualColumns virtualColumns + ) + { + JoinFilterPreAnalysisGroup group = new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + true + ); + + group.computeJoinFilterPreAnalysisIfAbsent( + originalFilter, + joinableClauses, + virtualColumns + ); + + return group; + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java index c58291301d06..ed6e1782f3a4 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/HashJoinSegmentStorageAdapterTest.java @@ -23,12 +23,10 @@ import com.google.common.collect.Lists; import org.apache.druid.common.config.NullHandling; 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.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; -import org.apache.druid.query.QueryContexts; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.OrDimFilter; @@ -37,8 +35,7 @@ import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.filter.SelectorFilter; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -301,21 +298,16 @@ public void test_makeCursors_factToCountryLeft() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -370,21 +362,16 @@ public void test_makeCursors_factToCountryLeftUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -437,21 +424,16 @@ public void test_makeCursors_factToCountryLeftUsingLookup() public void test_makeCursors_factToCountryInner() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.INNER)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -499,21 +481,16 @@ public void test_makeCursors_factToCountryInner() public void test_makeCursors_factToCountryInnerUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.INNER)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -563,21 +540,16 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumber() // is interpreted as 0 (a.k.a. Australia). List joinableClauses = ImmutableList.of(factToCountryOnNumber(JoinType.INNER)); Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -633,21 +605,16 @@ public void test_makeCursors_factToCountryInnerUsingCountryNumberUsingLookup() // is interpreted as 0 (a.k.a. Australia). List joinableClauses = ImmutableList.of(factToCountryNameUsingNumberLookup(JoinType.INNER)); Filter filter = new SelectorDimFilter("channel", "#en.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -699,21 +666,16 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFacts() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -740,21 +702,16 @@ public void test_makeCursors_factToCountryLeftWithFilterOnFactsUsingLookup() { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -780,21 +737,16 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNull() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -823,21 +775,16 @@ public void test_makeCursors_factToCountryRightWithFilterOnLeftIsNullUsingLookup { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.RIGHT)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -865,21 +812,16 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNull() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -908,21 +850,16 @@ public void test_makeCursors_factToCountryFullWithFilterOnLeftIsNullUsingLookup( { List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL)); Filter filter = new SelectorDimFilter("channel", null, null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -955,21 +892,16 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinable() null ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1002,21 +934,16 @@ public void test_makeCursors_factToCountryRightWithFilterOnJoinableUsingLookup() null ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1049,21 +976,16 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinable() new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber", "10", null) ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1096,21 +1018,16 @@ public void test_makeCursors_factToCountryLeftWithFilterOnJoinableUsingLookup() new SelectorDimFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Norway", null) ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1156,21 +1073,16 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit ExprMacroTable.nil() ).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1237,22 +1149,16 @@ public void test_makeCursors_factToCountryInnerWithFilterInsteadOfRealJoinCondit StringUtils.format("\"%sk\" == countryIsoCode", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX), ExprMacroTable.nil() ).toFilter(); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1302,22 +1208,16 @@ public void test_makeCursors_factToRegionToCountryLeft() factToRegion(JoinType.LEFT), regionToCountry(JoinType.LEFT) ); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1382,20 +1282,16 @@ public void test_makeCursors_factToCountryAlwaysTrue() ); Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1449,21 +1345,17 @@ public void test_makeCursors_factToCountryAlwaysFalse() Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1500,21 +1392,16 @@ public void test_makeCursors_factToCountryAlwaysTrueUsingLookup() Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1568,21 +1455,17 @@ public void test_makeCursors_factToCountryAlwaysFalseUsingLookup() Filter filter = new SelectorDimFilter("channel", "#de.wikipedia", null).toFilter(); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1628,21 +1511,16 @@ public void test_makeCursors_factToCountryUsingVirtualColumn() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + virtualColumns ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1694,21 +1572,16 @@ public void test_makeCursors_factToCountryUsingVirtualColumnUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + virtualColumns ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1752,21 +1625,16 @@ public void test_makeCursors_factToCountryUsingExpression() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1809,21 +1677,16 @@ public void test_makeCursors_factToCountryUsingExpressionUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1868,22 +1731,16 @@ public void test_makeCursors_factToRegionTheWrongWay() ); Filter filter = new SelectorDimFilter("regionIsoCode", "VA", null).toFilter(); - - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( filter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( filter, Intervals.ETERNITY, @@ -1929,21 +1786,17 @@ public void test_makeCursors_errorOnNonEquiJoin() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -1975,21 +1828,17 @@ public void test_makeCursors_errorOnNonEquiJoinUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2021,21 +1870,17 @@ public void test_makeCursors_errorOnNonKeyBasedJoin() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2066,21 +1911,17 @@ public void test_makeCursors_errorOnNonKeyBasedJoinUsingLookup() ) ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); JoinTestHelper.readCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( null, Intervals.ETERNITY, @@ -2099,21 +1940,16 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRows() Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( originalFilter, Intervals.ETERNITY, @@ -2138,21 +1974,16 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo { Filter originalFilter = new SelectorFilter("page", "this matches nothing"); List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.LEFT)); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses, + VirtualColumns.EMPTY ); - JoinTestHelper.verifyCursors( new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - preAnalysis + joinFilterPreAnalysisGroup ).makeCursors( originalFilter, Intervals.ETERNITY, @@ -2175,35 +2006,25 @@ public void test_makeCursors_factToCountryLeft_filterExcludesAllLeftRowsUsingLoo public void test_makeCursors_originalFilterDoesNotMatchPreAnalysis_shouldThrowISE() { List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.LEFT)); + Filter filter = new SelectorFilter("page", "this matches nothing"); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, + joinableClauses, + VirtualColumns.EMPTY + ); - JoinFilterPreAnalysis preAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( + new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), joinableClauses, + joinFilterPreAnalysisGroup + ).makeCursors( + filter, + Intervals.ETERNITY, VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + Granularities.ALL, + false, + null ); - Filter filter = new SelectorFilter("page", "this matches nothing"); - - try { - new HashJoinSegmentStorageAdapter( - factSegment.asStorageAdapter(), - joinableClauses, - preAnalysis - ).makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ); - Assert.fail(); - } - catch (ISE e) { - Assert.assertTrue(e.getMessage().startsWith("Filter provided to cursor [")); - } } + } 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 40a4863e9590..d04ebc83354a 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 @@ -20,29 +20,43 @@ package org.apache.druid.segment.join; import com.google.common.collect.ImmutableList; -import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.QueryContexts; +import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; -import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; +import org.apache.druid.segment.ReferenceCountingSegment; +import org.apache.druid.segment.SegmentReference; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.table.IndexedTableJoinable; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.hamcrest.CoreMatchers; +import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; import java.util.List; +import java.util.Optional; -public class HashJoinSegmentTest +public class HashJoinSegmentTest extends InitializedNullHandlingTest { + private JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ); + @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); @@ -50,17 +64,30 @@ public class HashJoinSegmentTest public ExpectedException expectedException = ExpectedException.none(); private QueryableIndexSegment baseSegment; + private ReferenceCountingSegment referencedSegment; private HashJoinSegment hashJoinSegment; - @BeforeClass - public static void setUpStatic() - { - NullHandling.initializeForTests(); - } + private int allReferencesAcquireCount; + private int allReferencesCloseCount; + private int referencedSegmentAcquireCount; + private int referencedSegmentClosedCount; + private int indexedTableJoinableReferenceAcquireCount; + private int indexedTableJoinableReferenceCloseCount; + private boolean j0Closed; + private boolean j1Closed; @Before public void setUp() throws IOException { + allReferencesAcquireCount = 0; + allReferencesCloseCount = 0; + referencedSegmentAcquireCount = 0; + referencedSegmentClosedCount = 0; + indexedTableJoinableReferenceAcquireCount = 0; + indexedTableJoinableReferenceCloseCount = 0; + j0Closed = false; + j1Closed = false; + baseSegment = new QueryableIndexSegment( JoinTestHelper.createFactIndexBuilder(temporaryFolder.newFolder()).buildMMappedIndex(), SegmentId.dummy("facts") @@ -69,33 +96,113 @@ public void setUp() throws IOException List joinableClauses = ImmutableList.of( new JoinableClause( "j0.", - new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()), + new IndexedTableJoinable(JoinTestHelper.createCountriesIndexedTable()) + { + @Override + public Optional acquireReferences() + { + if (!j0Closed) { + indexedTableJoinableReferenceAcquireCount++; + Closer closer = Closer.create(); + closer.register(() -> indexedTableJoinableReferenceCloseCount++); + return Optional.of(closer); + } + return Optional.empty(); + } + }, JoinType.LEFT, JoinConditionAnalysis.forExpression("1", "j0.", ExprMacroTable.nil()) ), new JoinableClause( "j1.", - new IndexedTableJoinable(JoinTestHelper.createRegionsIndexedTable()), + new IndexedTableJoinable(JoinTestHelper.createRegionsIndexedTable()) + { + @Override + public Optional acquireReferences() + { + if (!j1Closed) { + indexedTableJoinableReferenceAcquireCount++; + Closer closer = Closer.create(); + closer.register(() -> indexedTableJoinableReferenceCloseCount++); + return Optional.of(closer); + } + return Optional.empty(); + } + }, JoinType.LEFT, JoinConditionAnalysis.forExpression("1", "j1.", ExprMacroTable.nil()) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + true ); + referencedSegment = ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment); + SegmentReference testWrapper = new SegmentReference() + { + @Override + public Optional acquireReferences() + { + Closer closer = Closer.create(); + return referencedSegment.acquireReferences().map(closeable -> { + referencedSegmentAcquireCount++; + closer.register(closeable); + closer.register(() -> referencedSegmentClosedCount++); + return closer; + }); + } + + @Override + public SegmentId getId() + { + return referencedSegment.getId(); + } + + @Override + public Interval getDataInterval() + { + return referencedSegment.getDataInterval(); + } + + @Nullable + @Override + public QueryableIndex asQueryableIndex() + { + return referencedSegment.asQueryableIndex(); + } + + @Override + public StorageAdapter asStorageAdapter() + { + return referencedSegment.asStorageAdapter(); + } + + @Override + public void close() + { + referencedSegment.close(); + } + }; hashJoinSegment = new HashJoinSegment( - baseSegment, + testWrapper, joinableClauses, - joinFilterPreAnalysis - ); + joinFilterPreAnalysisGroup + ) + { + @Override + public Optional acquireReferences() + { + Closer closer = Closer.create(); + return super.acquireReferences().map(closeable -> { + allReferencesAcquireCount++; + closer.register(closeable); + closer.register(() -> allReferencesCloseCount++); + return closer; + }); + } + }; } @Test @@ -106,20 +213,15 @@ public void test_constructor_noClauses() List joinableClauses = ImmutableList.of(); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, - null, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + true ); final HashJoinSegment ignored = new HashJoinSegment( - baseSegment, + ReferenceCountingSegment.wrapRootGenerationSegment(baseSegment), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); } @@ -149,4 +251,79 @@ public void test_asStorageAdapter() CoreMatchers.instanceOf(HashJoinSegmentStorageAdapter.class) ); } + + @Test + public void testJoinableClausesAreClosedWhenReferencesUsed() throws IOException + { + Assert.assertFalse(referencedSegment.isClosed()); + + Optional maybeCloseable = hashJoinSegment.acquireReferences(); + Assert.assertTrue(maybeCloseable.isPresent()); + + Assert.assertEquals(1, referencedSegmentAcquireCount); + Assert.assertEquals(2, indexedTableJoinableReferenceAcquireCount); + Assert.assertEquals(1, allReferencesAcquireCount); + Assert.assertEquals(0, referencedSegmentClosedCount); + Assert.assertEquals(0, indexedTableJoinableReferenceCloseCount); + Assert.assertEquals(0, allReferencesCloseCount); + + Closeable closer = maybeCloseable.get(); + closer.close(); + + Assert.assertFalse(referencedSegment.isClosed()); + Assert.assertEquals(1, referencedSegmentClosedCount); + Assert.assertEquals(2, indexedTableJoinableReferenceCloseCount); + Assert.assertEquals(1, allReferencesCloseCount); + + } + + @Test + public void testJoinableClausesClosedIfSegmentIsAlreadyClosed() + { + Assert.assertFalse(referencedSegment.isClosed()); + + referencedSegment.close(); + Assert.assertTrue(referencedSegment.isClosed()); + + Optional maybeCloseable = hashJoinSegment.acquireReferences(); + Assert.assertFalse(maybeCloseable.isPresent()); + Assert.assertEquals(0, referencedSegmentAcquireCount); + Assert.assertEquals(0, indexedTableJoinableReferenceAcquireCount); + Assert.assertEquals(0, allReferencesAcquireCount); + Assert.assertEquals(0, referencedSegmentClosedCount); + Assert.assertEquals(0, indexedTableJoinableReferenceCloseCount); + Assert.assertEquals(0, allReferencesCloseCount); + } + + @Test + public void testJoinableClausesClosedIfJoinableZeroIsAlreadyClosed() + { + Assert.assertFalse(referencedSegment.isClosed()); + j0Closed = true; + + Optional maybeCloseable = hashJoinSegment.acquireReferences(); + Assert.assertFalse(maybeCloseable.isPresent()); + Assert.assertEquals(1, referencedSegmentAcquireCount); + Assert.assertEquals(0, indexedTableJoinableReferenceAcquireCount); + Assert.assertEquals(0, allReferencesAcquireCount); + Assert.assertEquals(1, referencedSegmentClosedCount); + Assert.assertEquals(0, indexedTableJoinableReferenceCloseCount); + Assert.assertEquals(0, allReferencesCloseCount); + } + + @Test + public void testJoinableClausesClosedIfJoinableOneIsAlreadyClosed() + { + Assert.assertFalse(referencedSegment.isClosed()); + j1Closed = true; + + Optional maybeCloseable = hashJoinSegment.acquireReferences(); + Assert.assertFalse(maybeCloseable.isPresent()); + Assert.assertEquals(1, referencedSegmentAcquireCount); + Assert.assertEquals(1, indexedTableJoinableReferenceAcquireCount); + Assert.assertEquals(0, allReferencesAcquireCount); + Assert.assertEquals(1, referencedSegmentClosedCount); + Assert.assertEquals(1, indexedTableJoinableReferenceCloseCount); + Assert.assertEquals(0, allReferencesCloseCount); + } } 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 753b35937891..893e0542b8ce 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 @@ -39,11 +39,15 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.filter.AndFilter; import org.apache.druid.segment.filter.BoundFilter; +import org.apache.druid.segment.filter.FalseFilter; import org.apache.druid.segment.filter.OrFilter; import org.apache.druid.segment.filter.SelectorFilter; import org.apache.druid.segment.join.filter.JoinFilterAnalyzer; import org.apache.druid.segment.join.filter.JoinFilterPreAnalysis; import org.apache.druid.segment.join.filter.JoinFilterSplit; +import org.apache.druid.segment.join.filter.JoinableClauses; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterPreAnalysisGroup; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.apache.druid.segment.join.table.IndexedTableJoinable; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -64,24 +68,17 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new SelectorFilter("channel", "#en.wikipedia"), - null, - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -114,6 +111,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannel() new Object[]{"History of Fourems", "Fourems Province", "Fourems"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("channel", "#en.wikipedia"), + null, + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -140,24 +146,17 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName regionExprToCountry ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("rtc.countryName", "United States"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -177,6 +176,15 @@ public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryName new Object[]{"Cream Soda", "Ainigriv", "United States"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("rtc.countryName", "United States"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -194,29 +202,18 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("US"), null, null).toFilter() - ) - ), - new SelectorFilter("rtc.countryName", "United States"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -240,6 +237,21 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelAndCount new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("US"), null, null).toFilter() + ) + ), + new SelectorFilter("rtc.countryName", "United States"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -259,33 +271,18 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("countryIsoCode", null), - new SelectorFilter("countryNumber", null), - new SelectorFilter("rtc.countryName", null), - new SelectorFilter("r1.regionName", null) - ) - ), - ImmutableSet.of() - ); - - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -310,8 +307,26 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnNullColumns() ) : ImmutableList.of() // when not running in SQL compatible mode, countryNumber does not have nulls ); - } + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("countryIsoCode", null), + new SelectorFilter("countryNumber", null), + new SelectorFilter("rtc.countryName", null), + new SelectorFilter("r1.regionName", null) + ) + ), + ImmutableSet.of() + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } + @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns() { @@ -329,30 +344,17 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new SelectorFilter("baseTableInvalidColumn", "abcd"), - new AndFilter( - ImmutableList.of( - new SelectorFilter("baseTableInvalidColumn2", null), - new SelectorFilter("rtc.invalidColumn", "abcd"), - new SelectorFilter("r1.invalidColumn", "abcd") - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -370,6 +372,22 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnInvalidColumns( ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("baseTableInvalidColumn", "abcd"), + new AndFilter( + ImmutableList.of( + new SelectorFilter("baseTableInvalidColumn2", null), + new SelectorFilter("rtc.invalidColumn", "abcd"), + new SelectorFilter("r1.invalidColumn", "abcd") + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -386,15 +404,15 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); VirtualColumns virtualColumns = VirtualColumns.create( @@ -408,14 +426,6 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC ) ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new SelectorFilter("v1", "virtual-column-#en.wikipedia"), - null, - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -447,6 +457,16 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnChannelVirtualC new Object[]{"History of Fourems", "Fourems Province", "Fourems"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new SelectorFilter("v1", "virtual-column-#en.wikipedia"), + null, + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -466,34 +486,21 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu ) ); - List joinableClauses = ImmutableList.of( + JoinableClauses joinableClauses = JoinableClauses.fromList(ImmutableList.of( factToRegion(JoinType.LEFT) - ); + )); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - virtualColumns, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses.getJoinableClauses(), + virtualColumns ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis + joinableClauses.getJoinableClauses(), + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("v0", "VIRGINIA"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -511,8 +518,17 @@ public void test_filterPushDown_factToRegionFilterOnRHSRegionNameExprVirtualColu new Object[]{"Old Anatolian Turkish", "VIRGINIA"} ) ); - } + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("v0", "VIRGINIA"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } @Test public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlreadyPushDownVariety() @@ -579,14 +595,34 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Les Argonautes", "Quebec", "Canada"} + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -653,27 +689,10 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterNormalizedAlready ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"Les Argonautes", "Quebec", "Canada"} - ) - ); } @Test @@ -703,15 +722,35 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan new SelectorFilter("rtc.countryName", "States United") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Old Anatolian Turkish", "Ainigriv", "States United"} + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -724,6 +763,8 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan new SelectorFilter("rtc.countryName", "States United"), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( @@ -743,25 +784,6 @@ public void test_filterPushDown_factExpressionsToRegionToCountryLeftFilterOnChan ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() .iterator().next(); compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"Old Anatolian Turkish", "Ainigriv", "States United"} - ) - ); } @Test @@ -793,9 +815,10 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); expectedException.expect(IllegalArgumentException.class); expectedException.expectMessage("Cannot build hash-join matcher on non-equi-join condition: \"r1.regionIsoCode\" == regionIsoCode && reverse(\"r1.countryIsoCode\") == countryIsoCode"); @@ -803,7 +826,7 @@ public void test_filterPushDown_factToRegionToCountryNotEquiJoinLeftFilterOnChan HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -844,18 +867,39 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"유희왕 GX", "Seoul", "Republic of Korea"}, + new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} + ) + ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( ImmutableList.of( new OrFilter( ImmutableList.of( @@ -894,28 +938,11 @@ public void test_filterPushDown_factToRegionToCountryLeftUnnormalizedFilter() ), ImmutableSet.of() ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"유희왕 GX", "Seoul", "Republic of Korea"}, - new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} - ) - ); } @Test @@ -944,15 +971,35 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + filter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"President of India", "Usca"}, + new Object[]{"Otjiwarongo Airport", "Usca"}, + new Object[]{"Carlo Curti", "Usca"} + ) ); ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( @@ -973,6 +1020,8 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel expectedVirtualColumn ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals( expectedFilterSplit.getBaseTableFilter(), @@ -985,26 +1034,6 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() .iterator().next(); compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"President of India", "Usca"}, - new Object[]{"Otjiwarongo Airport", "Usca"}, - new Object[]{"Carlo Curti", "Usca"} - ) - ); } @Test @@ -1033,15 +1062,35 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + filter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" + ), + ImmutableList.of( + new Object[]{"President of India", "Usca"}, + new Object[]{"Otjiwarongo Airport", "Usca"}, + new Object[]{"Carlo Curti", "Usca"} + ) ); ExpressionVirtualColumn expectedVirtualColumn = new ExpressionVirtualColumn( @@ -1062,6 +1111,8 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel expectedVirtualColumn ) ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals( expectedFilterSplit.getBaseTableFilter(), @@ -1074,26 +1125,6 @@ public void test_filterPushDown_factConcatExpressionToCountryLeftFilterOnChannel ExpressionVirtualColumn actualVirtualColumn = (ExpressionVirtualColumn) actualFilterSplit.getPushDownVirtualColumns() .iterator().next(); compareExpressionVirtualColumns(expectedVirtualColumn, actualVirtualColumn); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - filter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" - ), - ImmutableList.of( - new Object[]{"President of India", "Usca"}, - new Object[]{"Otjiwarongo Airport", "Usca"}, - new Object[]{"Carlo Curti", "Usca"} - ) - ); } @Test @@ -1106,29 +1137,17 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#de.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1150,6 +1169,21 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new Object[]{"Diskussion:Sebastian Schulz", "DE", 3L, "DE", "Germany", 3L} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#de.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "Germany"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1162,29 +1196,17 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#de.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( originalFilter, @@ -1205,6 +1227,21 @@ public void test_filterPushDown_factToCountryRightWithFilterOnChannelAndJoinable new Object[]{"Diskussion:Sebastian Schulz", "DE", 3L, "DE", "Germany"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#de.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("DE"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "Germany"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1217,14 +1254,35 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + "countryIsoCode", + "countryNumber", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryIsoCode", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", + FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryNumber" + ), + ImmutableList.of() ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -1237,8 +1295,32 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } + + @Test + public void test_filterPushDown_factToCountryRightWithFilterOnValueThatMatchesNothing() + { + List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.RIGHT)); + Filter originalFilter = new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "NO MATCH") + ) + ); + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysisGroup + ); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1259,6 +1341,22 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumns() ), ImmutableList.of() ); + + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + FalseFilter.instance(), + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "NO MATCH") + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1271,28 +1369,16 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1310,8 +1396,23 @@ public void test_filterPushDown_factToCountryRightWithFilterOnNullColumnsUsingLo FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "k", FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v" ), - ImmutableList.of() + ImmutableList.of() + ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) + ) + ), + ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1324,28 +1425,16 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); // 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). @@ -1374,6 +1463,21 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new Object[]{"Peremptory norm", "AU", "AU", "Australia", 0L} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", "Australia"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1386,28 +1490,16 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia") ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#en.wikipedia"), - new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); // 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). @@ -1435,6 +1527,21 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnChan new Object[]{"Peremptory norm", "AU", "0", "Australia"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#en.wikipedia"), + new InDimFilter("countryNumber", ImmutableSet.of("0"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", "Australia"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1447,28 +1554,16 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1488,6 +1583,21 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "countryName", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1500,28 +1610,16 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null) - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1540,6 +1638,21 @@ public void test_filterPushDown_factToCountryInnerUsingCountryNumberFilterOnNull ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_NUMBER_PREFIX + "v", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1552,28 +1665,16 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa ) ); List joinableClauses = ImmutableList.of(factToCountryOnIsoCode(JoinType.FULL)); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#es.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "El Salvador"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1596,6 +1697,21 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa new Object[]{"Wendigo", "SV", 12L, "SV", "El Salvador", 12L} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#es.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", "El Salvador"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1608,29 +1724,17 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa ) ); List joinableClauses = ImmutableList.of(factToCountryNameUsingIsoCodeLookup(JoinType.FULL)); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + filter, joinableClauses, - filter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", "#es.wikipedia"), - new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() - ) - ), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "El Salvador"), - ImmutableSet.of() - ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - JoinTestHelper.verifyCursors( adapter.makeCursors( filter, @@ -1651,6 +1755,21 @@ public void test_filterPushDown_factToCountryFullWithFilterOnChannelAndCountryNa new Object[]{"Wendigo", "SV", 12L, "SV", "El Salvador"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", "#es.wikipedia"), + new InDimFilter("countryIsoCode", ImmutableSet.of("SV"), null, null).toFilter() + ) + ), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", "El Salvador"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1663,28 +1782,16 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1705,6 +1812,21 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNulls() ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "countryName", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1717,28 +1839,16 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new AndFilter( - ImmutableList.of( - new SelectorFilter("channel", null), - new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) - ) - ), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1758,6 +1868,21 @@ public void test_filterPushDown_factToCountryFullWithFilterOnNullsUsingLookup() ), ImmutableList.of() ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new AndFilter( + ImmutableList.of( + new SelectorFilter("channel", null), + new SelectorFilter(FACT_TO_COUNTRY_ON_ISO_CODE_PREFIX + "v", null) + ) + ), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1782,29 +1907,16 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR ); Filter originalFilter = new SelectorFilter("r1.regionName", "Fourems Province"); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new AndFilter( - ImmutableList.of( - new InDimFilter("countryIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), - new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter() - ) - ), - new SelectorFilter("r1.regionName", "Fourems Province"), - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1823,6 +1935,22 @@ public void test_filterPushDown_factToRegionTwoColumnsToOneRHSColumnAndFilterOnR new Object[]{"History of Fourems", "Fourems Province"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new AndFilter( + ImmutableList.of( + new InDimFilter("countryIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), + new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter() + ) + ), + new SelectorFilter("r1.regionName", "Fourems Province"), + ImmutableSet.of() + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -1852,28 +1980,16 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new OrFilter( - ImmutableList.of( - new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), - new SelectorFilter("regionIsoCode", "AAAA") - ) - ), - originalFilter, - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -1892,6 +2008,22 @@ public void test_filterPushDown_factToRegionOneColumnToTwoRHSColumnsAndFilterOnR new Object[]{"History of Fourems", "Fourems Province"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new OrFilter( + ImmutableList.of( + new InDimFilter("regionIsoCode", ImmutableSet.of("MMMM"), null, null).toFilter(), + new SelectorFilter("regionIsoCode", "AAAA") + ) + ), + originalFilter, + ImmutableSet.of() + ); + + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @@ -1923,28 +2055,16 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - new OrFilter( - ImmutableList.of( - new SelectorFilter("user", "Fourems Province"), - new SelectorFilter("regionIsoCode", "AAAA") - ) - ), - null, - ImmutableSet.of() + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); // This query doesn't execute because regionName is not a key column, but we can still check the // filter rewrites. @@ -1968,40 +2088,52 @@ public void test_filterPushDown_factToRegionThreeRHSColumnsAllDirectAndFilterOnR ), ImmutableList.of() ); - } + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + new OrFilter( + ImmutableList.of( + new SelectorFilter("user", "Fourems Province"), + new SelectorFilter("regionIsoCode", "AAAA") + ) + ), + null, + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } @Test public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePushDown() { - List joinableClauses = ImmutableList.of( + JoinableClauses joinableClauses = JoinableClauses.fromList(ImmutableList.of( factToRegion(JoinType.LEFT), regionToCountry(JoinType.LEFT) - ); + )); Filter originalFilter = new SelectorFilter("page", "Peremptory norm"); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + false, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ), + true + ); + joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( originalFilter, - false, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses.getJoinableClauses(), + VirtualColumns.EMPTY ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis - ); - - JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( - null, - new SelectorFilter("page", "Peremptory norm"), - ImmutableSet.of() + joinableClauses.getJoinableClauses(), + joinFilterPreAnalysisGroup ); - JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); - Assert.assertEquals(expectedFilterSplit, actualFilterSplit); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -2021,15 +2153,25 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnPageDisablePush new Object[]{"Peremptory norm", "New South Wales", "Australia"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("page", "Peremptory norm"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRewrite() { - List joinableClauses = ImmutableList.of( + JoinableClauses joinableClauses = JoinableClauses.fromList(ImmutableList.of( factToRegion(JoinType.LEFT), regionToCountry(JoinType.LEFT) - ); + )); Filter originalFilter = new AndFilter( ImmutableList.of( new SelectorFilter("channel", "#en.wikipedia"), @@ -2052,19 +2194,48 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe )) ) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + true, + false, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ), + true + ); + joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( originalFilter, - true, - false, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + joinableClauses.getJoinableClauses(), + VirtualColumns.EMPTY ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), - joinableClauses, - joinFilterPreAnalysis + joinableClauses.getJoinableClauses(), + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"President of India", "California", "United States"}, + new Object[]{"Otjiwarongo Airport", "California", "United States"}, + new Object[]{"DirecTV", "North Carolina", "United States"}, + new Object[]{"Carlo Curti", "California", "United States"}, + new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -2096,31 +2267,10 @@ public void test_filterPushDown_factToRegionToCountryLeftEnablePushDownDisableRe ), ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"President of India", "California", "United States"}, - new Object[]{"Otjiwarongo Airport", "California", "United States"}, - new Object[]{"DirecTV", "North Carolina", "United States"}, - new Object[]{"Carlo Curti", "California", "United States"}, - new Object[]{"Old Anatolian Turkish", "Virginia", "United States"} - ) - ); } @Test @@ -2228,15 +2378,15 @@ public boolean supportsRequiredColumnRewrite() regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); - HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup ); String rewrittenCountryIsoCodeColumnName = hasLhsExpressionInJoinCondition @@ -2268,6 +2418,25 @@ public boolean supportsRequiredColumnRewrite() expectedVirtualColumns = ImmutableSet.of(); } + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + new Object[]{"Didier Leclair", "Ontario", "Canada"} + ) + ); + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( new AndFilter( ImmutableList.of( @@ -2345,27 +2514,10 @@ public boolean supportsRequiredColumnRewrite() ), expectedVirtualColumns ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); - - JoinTestHelper.verifyCursors( - adapter.makeCursors( - originalFilter, - Intervals.ETERNITY, - VirtualColumns.EMPTY, - Granularities.ALL, - false, - null - ), - ImmutableList.of( - "page", - FACT_TO_REGION_PREFIX + "regionName", - REGION_TO_COUNTRY_PREFIX + "countryName" - ), - ImmutableList.of( - new Object[]{"Didier Leclair", "Ontario", "Canada"} - ) - ); } @Test @@ -2383,14 +2535,34 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa regionToCountry(JoinType.LEFT) ); - JoinFilterPreAnalysis joinFilterPreAnalysis = simplePreAnalysis( + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = makeDefaultConfigPreAnalysisGroup( + originalFilter, joinableClauses, - originalFilter + VirtualColumns.EMPTY ); + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( factSegment.asStorageAdapter(), joinableClauses, - joinFilterPreAnalysis + joinFilterPreAnalysisGroup + ); + + JoinTestHelper.verifyCursors( + adapter.makeCursors( + originalFilter, + Intervals.ETERNITY, + VirtualColumns.EMPTY, + Granularities.ALL, + false, + null + ), + ImmutableList.of( + "page", + FACT_TO_REGION_PREFIX + "regionName", + REGION_TO_COUNTRY_PREFIX + "countryName" + ), + ImmutableList.of( + ) ); JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( @@ -2413,8 +2585,57 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa originalFilter, ImmutableSet.of() ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getPreAnalysisForSingleLevelMode(); + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); Assert.assertEquals(expectedFilterSplit, actualFilterSplit); + } + + + @Test + public void test_filterPushDown_factToRegionExprToCountryLeftFilterOnCountryNameWithMultiLevelMode() + { + Filter originalFilter = new SelectorFilter("rtc.countryName", "United States"); + JoinableClause regionExprToCountry = new JoinableClause( + REGION_TO_COUNTRY_PREFIX, + new IndexedTableJoinable(countriesTable), + JoinType.LEFT, + JoinConditionAnalysis.forExpression( + StringUtils.format( + "reverse(\"%scountryIsoCode\") == \"%scountryIsoCode\"", + FACT_TO_REGION_PREFIX, + REGION_TO_COUNTRY_PREFIX + ), + REGION_TO_COUNTRY_PREFIX, + ExprMacroTable.nil() + ) + ); + List joinableClauses = ImmutableList.of( + factToRegion(JoinType.LEFT), + regionExprToCountry + ); + + JoinFilterPreAnalysisGroup joinFilterPreAnalysisGroup = new JoinFilterPreAnalysisGroup( + new JoinFilterRewriteConfig( + true, + true, + true, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ), + false + ); + + joinFilterPreAnalysisGroup.computeJoinFilterPreAnalysisIfAbsent( + originalFilter, + joinableClauses, + VirtualColumns.EMPTY + ); + + HashJoinSegmentStorageAdapter adapter = new HashJoinSegmentStorageAdapter( + factSegment.asStorageAdapter(), + joinableClauses, + joinFilterPreAnalysisGroup + ); JoinTestHelper.verifyCursors( adapter.makeCursors( @@ -2431,8 +2652,22 @@ public void test_filterPushDown_factToRegionToCountryLeftFilterOnTwoRHSColumnsSa REGION_TO_COUNTRY_PREFIX + "countryName" ), ImmutableList.of( + new Object[]{"Cream Soda", "Ainigriv", "United States"} ) ); + + JoinFilterSplit expectedFilterSplit = new JoinFilterSplit( + null, + new SelectorFilter("rtc.countryName", "United States"), + ImmutableSet.of() + ); + JoinFilterPreAnalysis joinFilterPreAnalysis = joinFilterPreAnalysisGroup.getAnalysis( + originalFilter, + VirtualColumns.EMPTY + ); + + JoinFilterSplit actualFilterSplit = JoinFilterAnalyzer.splitFilter(joinFilterPreAnalysis); + Assert.assertEquals(expectedFilterSplit, actualFilterSplit); } @Test @@ -2444,19 +2679,13 @@ public void test_JoinFilterSplit_equals() .verify(); } - private static JoinFilterPreAnalysis simplePreAnalysis( - List joinableClauses, - Filter originalFilter - ) + + @Test + public void test_JoinFilterPreAnalysisGroupKey_equals() { - return JoinFilterAnalyzer.computeJoinFilterPreAnalysis( - joinableClauses, - VirtualColumns.EMPTY, - originalFilter, - true, - true, - true, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE - ); + EqualsVerifier.forClass(JoinFilterPreAnalysisGroup.JoinFilterPreAnalysisGroupKey.class) + .usingGetClass() + .withNonnullFields("filter", "virtualColumns") + .verify(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index 0f960dee5465..d33e2627a344 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -142,6 +142,8 @@ public Supplier makeComplexProcessor(BaseObjectColumnValueSelector se } }; + public static final String INDEXED_TABLE_VERSION = DateTimes.nowUtc().toString(); + private static RowAdapter> createMapRowAdapter(final RowSignature signature) { return new RowAdapter>() @@ -255,7 +257,8 @@ public static RowBasedIndexedTable> createCountriesIndexedTa rows, createMapRowAdapter(COUNTRIES_SIGNATURE), COUNTRIES_SIGNATURE, - ImmutableSet.of("countryNumber", "countryIsoCode") + ImmutableSet.of("countryNumber", "countryIsoCode"), + INDEXED_TABLE_VERSION ) ); } @@ -268,7 +271,8 @@ public static RowBasedIndexedTable> createRegionsIndexedTabl rows, createMapRowAdapter(REGIONS_SIGNATURE), REGIONS_SIGNATURE, - ImmutableSet.of("regionIsoCode", "countryIsoCode") + ImmutableSet.of("regionIsoCode", "countryIsoCode"), + INDEXED_TABLE_VERSION ) ); } diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java b/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java index 4fa521dbcbc9..c418f7fd45ef 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinablesTest.java @@ -22,21 +22,34 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.LookupDataSource; +import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.TableDataSource; +import org.apache.druid.query.TestQuery; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.planning.PreJoinableClause; -import org.apache.druid.segment.Segment; -import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.join.lookup.LookupJoinable; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; import java.util.Optional; import java.util.concurrent.atomic.AtomicLong; @@ -44,6 +57,13 @@ public class JoinablesTest { + private static final JoinFilterRewriteConfig DEFAULT_JOIN_FILTER_REWRITE_CONFIG = new JoinFilterRewriteConfig( + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, + QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE + ); + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -95,16 +115,12 @@ public void test_isPrefixedBy() @Test public void test_createSegmentMapFn_noClauses() { - final Function segmentMapFn = Joinables.createSegmentMapFn( + final Function segmentMapFn = Joinables.createSegmentMapFn( ImmutableList.of(), NoopJoinableFactory.INSTANCE, new AtomicLong(), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, - null, - VirtualColumns.EMPTY + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + null ); Assert.assertSame(Function.identity(), segmentMapFn); @@ -124,16 +140,12 @@ public void test_createSegmentMapFn_unusableClause() expectedException.expect(IllegalStateException.class); expectedException.expectMessage("dataSource is not joinable"); - final Function ignored = Joinables.createSegmentMapFn( + final Function ignored = Joinables.createSegmentMapFn( ImmutableList.of(clause), NoopJoinableFactory.INSTANCE, new AtomicLong(), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, - null, - VirtualColumns.EMPTY + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + null ); } @@ -153,24 +165,36 @@ public void test_createSegmentMapFn_usableClause() conditionAnalysis ); - final Function segmentMapFn = Joinables.createSegmentMapFn( + final Function segmentMapFn = Joinables.createSegmentMapFn( ImmutableList.of(clause), - (dataSource, condition) -> { - if (dataSource.equals(lookupDataSource) && condition.equals(conditionAnalysis)) { - return Optional.of( - LookupJoinable.wrap(new MapLookupExtractor(ImmutableMap.of("k", "v"), false)) - ); - } else { - return Optional.empty(); + new JoinableFactory() + { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return dataSource.equals(lookupDataSource); + } + + @Override + public Optional build(DataSource dataSource, JoinConditionAnalysis condition) + { + if (dataSource.equals(lookupDataSource) && condition.equals(conditionAnalysis)) { + return Optional.of( + LookupJoinable.wrap(new MapLookupExtractor(ImmutableMap.of("k", "v"), false)) + ); + } else { + return Optional.empty(); + } } }, new AtomicLong(), - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_PUSH_DOWN, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS, - QueryContexts.DEFAULT_ENABLE_JOIN_FILTER_REWRITE_MAX_SIZE, - null, - VirtualColumns.EMPTY + DEFAULT_JOIN_FILTER_REWRITE_CONFIG, + new TestQuery( + new TableDataSource("test"), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("0/100"))), + false, + new HashMap() + ) ); Assert.assertNotSame(Function.identity(), segmentMapFn); @@ -224,4 +248,83 @@ public void test_checkClausePrefixesForDuplicatesAndShadowing_shadowing() Joinables.checkPrefixesForDuplicatesAndShadowing(prefixes); } + + @Test + public void test_gatherAllJoinQueryLevels() + { + Query query1 = new GroupByQuery.Builder() + .addDimension("dim1") + .setDataSource( + JoinDataSource.create( + new NoopDataSource(), + new NoopDataSource(), + "111", + "1", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + Query query2 = new GroupByQuery.Builder() + .addDimension("dim2") + .setDataSource( + JoinDataSource.create( + new NoopDataSource(), + new NoopDataSource(), + "222", + "1", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + QueryDataSource queryDataSource1 = new QueryDataSource(query1); + QueryDataSource queryDataSource2 = new QueryDataSource(query2); + + Query query3 = new GroupByQuery.Builder() + .addDimension("dim2") + .setDataSource( + JoinDataSource.create( + JoinDataSource.create( + queryDataSource1, + new NoopDataSource(), + "444", + "4", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ), + queryDataSource2, + "333", + "3", + JoinType.LEFT, + TestExprMacroTable.INSTANCE + ) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + Query queryOuter = new GroupByQuery.Builder() + .addDimension("dim") + .setDataSource( + new QueryDataSource(query3) + ) + .setInterval("1999/2000") + .setGranularity(Granularities.YEAR) + .build(); + + List joinQueryLevels = new ArrayList<>(); + Joinables.gatherAllJoinQueryLevels(queryOuter, joinQueryLevels); + + Assert.assertEquals( + ImmutableList.of(query3, query1, query2), + joinQueryLevels + ); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/MapJoinableFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/join/MapJoinableFactoryTest.java index cf0336061409..1d00e711c190 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/MapJoinableFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/MapJoinableFactoryTest.java @@ -65,6 +65,8 @@ public void setUp() target = new MapJoinableFactory( ImmutableMap.of(NoopDataSource.class, noopJoinableFactory)); } + + @Test public void testBuildDataSourceNotRegisteredShouldReturnAbsent() { @@ -89,4 +91,18 @@ public void testBuildDataSourceIsRegisteredShouldReturnJoinableFromFactory() Optional joinable = target.build(noopDataSource, condition); Assert.assertEquals(mockJoinable, joinable.get()); } + + @Test + public void testIsDirectShouldBeFalseForNotRegistered() + { + Assert.assertFalse(target.isDirectlyJoinable(inlineDataSource)); + } + + @Test + public void testIsDirectlyJoinableShouldBeTrueForRegisteredThatIsJoinable() + { + EasyMock.expect(noopJoinableFactory.isDirectlyJoinable(noopDataSource)).andReturn(true).anyTimes(); + EasyMock.replay(noopJoinableFactory); + Assert.assertTrue(target.isDirectlyJoinable(noopDataSource)); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java b/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java index ff138041f131..1583b02b5d5d 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java +++ b/processing/src/test/java/org/apache/druid/segment/join/NoopJoinableFactory.java @@ -32,6 +32,12 @@ private NoopJoinableFactory() // Singleton. } + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return false; + } + @Override public Optional build(DataSource dataSource, JoinConditionAnalysis condition) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java b/processing/src/test/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysisTest.java similarity index 56% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java rename to processing/src/test/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysisTest.java index 2bc00ce3d9b6..39fe2bcefb5a 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIOConfig.java +++ b/processing/src/test/java/org/apache/druid/segment/join/filter/JoinFilterColumnCorrelationAnalysisTest.java @@ -17,24 +17,16 @@ * under the License. */ -package org.apache.druid.indexing.common.task.batch.parallel; +package org.apache.druid.segment.join.filter; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import org.apache.druid.segment.indexing.IOConfig; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; -import java.util.List; - -@JsonTypeName(PartialHashSegmentMergeTask.TYPE) -class PartialHashSegmentMergeIOConfig extends PartialSegmentMergeIOConfig - implements IOConfig +public class JoinFilterColumnCorrelationAnalysisTest { - @JsonCreator - PartialHashSegmentMergeIOConfig( - @JsonProperty("partitionLocations") List partitionLocations - ) + @Test + public void testEqualsContract() { - super(partitionLocations); + EqualsVerifier.forClass(JoinFilterColumnCorrelationAnalysis.class).usingGetClass().verify(); } } diff --git a/processing/src/main/java/org/apache/druid/query/LegacyDataSource.java b/processing/src/test/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandateTest.java similarity index 74% rename from processing/src/main/java/org/apache/druid/query/LegacyDataSource.java rename to processing/src/test/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandateTest.java index d3812489be20..2586307008b2 100644 --- a/processing/src/main/java/org/apache/druid/query/LegacyDataSource.java +++ b/processing/src/test/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandateTest.java @@ -17,17 +17,16 @@ * under the License. */ -package org.apache.druid.query; +package org.apache.druid.segment.join.filter.rewrite; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonTypeName; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; -@JsonTypeName("table") -public class LegacyDataSource extends TableDataSource +public class RhsRewriteCandateTest { - @JsonCreator - public LegacyDataSource(String name) + @Test + public void testEqualsContract() { - super(name); + EqualsVerifier.forClass(RhsRewriteCandidate.class).usingGetClass().verify(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java b/processing/src/test/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidatesTest.java similarity index 55% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java rename to processing/src/test/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidatesTest.java index abfef7608809..331a6a66600b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialHashSegmentMergeIngestionSpec.java +++ b/processing/src/test/java/org/apache/druid/segment/join/filter/rewrite/RhsRewriteCandidatesTest.java @@ -17,21 +17,16 @@ * under the License. */ -package org.apache.druid.indexing.common.task.batch.parallel; +package org.apache.druid.segment.join.filter.rewrite; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.segment.indexing.DataSchema; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.junit.Test; -class PartialHashSegmentMergeIngestionSpec extends PartialSegmentMergeIngestionSpec +public class RhsRewriteCandidatesTest { - @JsonCreator - PartialHashSegmentMergeIngestionSpec( - @JsonProperty("dataSchema") DataSchema dataSchema, - @JsonProperty("ioConfig") PartialHashSegmentMergeIOConfig ioConfig, - @JsonProperty("tuningConfig") ParallelIndexTuningConfig tuningConfig - ) + @Test + public void testEqualsContract() { - super(dataSchema, ioConfig, tuningConfig); + EqualsVerifier.forClass(RhsRewriteCandidates.class).usingGetClass().verify(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java b/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java index 4115b84acec2..2037f7763172 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/lookup/LookupJoinableTest.java @@ -36,6 +36,7 @@ import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.Set; @RunWith(MockitoJUnitRunner.class) @@ -117,7 +118,7 @@ public void getColumnCapabilitiesForUnknownColumnShouldReturnNull() @Test public void getCorrelatedColummnValuesMissingSearchColumnShouldReturnEmptySet() { - Set correlatedValues = + Optional> correlatedValues = target.getCorrelatedColumnValues( UNKNOWN_COLUMN, SEARCH_KEY_VALUE, @@ -125,13 +126,13 @@ public void getCorrelatedColummnValuesMissingSearchColumnShouldReturnEmptySet() 0, false); - Assert.assertEquals(Collections.emptySet(), correlatedValues); + Assert.assertFalse(correlatedValues.isPresent()); } @Test public void getCorrelatedColummnValuesMissingRetrievalColumnShouldReturnEmptySet() { - Set correlatedValues = + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.KEY_COLUMN, SEARCH_KEY_VALUE, @@ -139,85 +140,85 @@ public void getCorrelatedColummnValuesMissingRetrievalColumnShouldReturnEmptySet 0, false); - Assert.assertEquals(Collections.emptySet(), correlatedValues); + Assert.assertFalse(correlatedValues.isPresent()); } @Test public void getCorrelatedColumnValuesForSearchKeyAndRetrieveKeyColumnShouldReturnSearchValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.KEY_COLUMN, SEARCH_KEY_VALUE, LookupColumnSelectorFactory.KEY_COLUMN, 0, false); - Assert.assertEquals(ImmutableSet.of(SEARCH_KEY_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_KEY_VALUE)), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchKeyAndRetrieveValueColumnShouldReturnExtractedValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.KEY_COLUMN, SEARCH_KEY_VALUE, LookupColumnSelectorFactory.VALUE_COLUMN, 0, false); - Assert.assertEquals(ImmutableSet.of(SEARCH_VALUE_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_VALUE_VALUE)), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchKeyMissingAndRetrieveValueColumnShouldReturnExtractedValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.KEY_COLUMN, SEARCH_KEY_NULL_VALUE, LookupColumnSelectorFactory.VALUE_COLUMN, 0, false); - Assert.assertEquals(Collections.singleton(null), correlatedValues); + Assert.assertEquals(Optional.of(Collections.singleton(null)), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchValueAndRetrieveValueColumnAndNonKeyColumnSearchDisabledShouldReturnSearchValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.VALUE_COLUMN, SEARCH_VALUE_VALUE, LookupColumnSelectorFactory.VALUE_COLUMN, 10, false); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.VALUE_COLUMN, SEARCH_VALUE_VALUE, LookupColumnSelectorFactory.KEY_COLUMN, 10, false); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchValueAndRetrieveValueColumnShouldReturnSearchValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.VALUE_COLUMN, SEARCH_VALUE_VALUE, LookupColumnSelectorFactory.VALUE_COLUMN, 0, true); - Assert.assertEquals(ImmutableSet.of(SEARCH_VALUE_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_VALUE_VALUE)), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchValueAndRetrieveKeyColumnShouldReturnUnAppliedValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.VALUE_COLUMN, SEARCH_VALUE_VALUE, LookupColumnSelectorFactory.KEY_COLUMN, 10, true); - Assert.assertEquals(ImmutableSet.of(SEARCH_KEY_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_KEY_VALUE)), correlatedValues); } @Test @@ -228,24 +229,24 @@ public void getCorrelatedColumnValuesForSearchValueAndRetrieveKeyColumnShouldRet */ public void getCorrelatedColumnValuesForSearchValueAndRetrieveKeyColumnWithMaxLimitSetShouldHonorMaxLimit() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.VALUE_COLUMN, SEARCH_VALUE_VALUE, LookupColumnSelectorFactory.KEY_COLUMN, 0, true); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchUnknownValueAndRetrieveKeyColumnShouldReturnNoCorrelatedValues() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( LookupColumnSelectorFactory.VALUE_COLUMN, SEARCH_VALUE_UNKNOWN, LookupColumnSelectorFactory.KEY_COLUMN, 10, true); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of()), correlatedValues); } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java index e7baaea1edd7..45367b2374e8 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinMatcherTest.java @@ -23,14 +23,20 @@ import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.ints.IntList; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.segment.ConstantDimensionSelector; import org.apache.druid.segment.DimensionDictionarySelector; +import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.data.ArrayBasedIndexedInts; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.experimental.runners.Enclosed; import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.MockitoAnnotations; import java.util.Collections; import java.util.concurrent.atomic.AtomicLong; @@ -48,12 +54,89 @@ public static class ConditionMatcherFactoryTest { public static class MakeDimensionProcessorTest { + @Mock + private DimensionSelector dimensionSelector; + private static final String KEY = "key"; static { NullHandling.initializeForTests(); } + @SuppressWarnings("ReturnValueIgnored") + @Test(expected = QueryUnsupportedException.class) + public void testMatchMultiValuedRowCardinalityUnknownShouldThrowException() + { + MockitoAnnotations.initMocks(this); + ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{2, 4, 6}); + Mockito.doReturn(row).when(dimensionSelector).getRow(); + Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality(); + + IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = + new IndexedTableJoinMatcher.ConditionMatcherFactory( + ValueType.STRING, + IndexedTableJoinMatcherTest::createSingletonIntList + ); + Supplier dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); + // Test match should throw exception + dimensionProcessor.get(); + } + + @SuppressWarnings("ReturnValueIgnored") + @Test(expected = QueryUnsupportedException.class) + public void testMatchMultiValuedRowCardinalityKnownShouldThrowException() + { + MockitoAnnotations.initMocks(this); + ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{2, 4, 6}); + Mockito.doReturn(row).when(dimensionSelector).getRow(); + Mockito.doReturn(3).when(dimensionSelector).getValueCardinality(); + + IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = + new IndexedTableJoinMatcher.ConditionMatcherFactory( + ValueType.STRING, + IndexedTableJoinMatcherTest::createSingletonIntList + ); + Supplier dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); + // Test match should throw exception + dimensionProcessor.get(); + } + + @Test + public void testMatchEmptyRowCardinalityUnknown() + { + MockitoAnnotations.initMocks(this); + ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{}); + Mockito.doReturn(row).when(dimensionSelector).getRow(); + Mockito.doReturn(DimensionDictionarySelector.CARDINALITY_UNKNOWN).when(dimensionSelector).getValueCardinality(); + + IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = + new IndexedTableJoinMatcher.ConditionMatcherFactory( + ValueType.STRING, + IndexedTableJoinMatcherTest::createSingletonIntList + ); + Supplier dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); + Assert.assertNotNull(dimensionProcessor.get()); + Assert.assertFalse(dimensionProcessor.get().hasNext()); + } + + @Test + public void testMatchEmptyRowCardinalityKnown() + { + MockitoAnnotations.initMocks(this); + ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{}); + Mockito.doReturn(row).when(dimensionSelector).getRow(); + Mockito.doReturn(0).when(dimensionSelector).getValueCardinality(); + + IndexedTableJoinMatcher.ConditionMatcherFactory conditionMatcherFactory = + new IndexedTableJoinMatcher.ConditionMatcherFactory( + ValueType.STRING, + IndexedTableJoinMatcherTest::createSingletonIntList + ); + Supplier dimensionProcessor = conditionMatcherFactory.makeDimensionProcessor(dimensionSelector, false); + Assert.assertNotNull(dimensionProcessor.get()); + Assert.assertFalse(dimensionProcessor.get().hasNext()); + } + @Test public void getsCorrectResultWhenSelectorCardinalityUnknown() { diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java index 89f78dc523d2..c75232c9be9b 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/IndexedTableJoinableTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.dimension.DefaultDimensionSpec; @@ -40,6 +41,7 @@ import org.junit.Test; import java.util.Collections; +import java.util.Optional; import java.util.Set; public class IndexedTableJoinableTest @@ -95,7 +97,8 @@ public ColumnCapabilities getColumnCapabilities(String columnName) inlineDataSource.getRowsAsList(), inlineDataSource.rowAdapter(), inlineDataSource.getRowSignature(), - ImmutableSet.of("str") + ImmutableSet.of("str"), + DateTimes.nowUtc().toString() ); private IndexedTableJoinable target; @@ -136,9 +139,8 @@ public void getColumnCapabilitiesForStringColumn() Assert.assertEquals(ValueType.STRING, capabilities.getType()); Assert.assertTrue(capabilities.isDictionaryEncoded()); Assert.assertFalse(capabilities.hasBitmapIndexes()); - Assert.assertFalse(capabilities.hasMultipleValues()); + Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(capabilities.hasSpatialIndexes()); - Assert.assertTrue(capabilities.isComplete()); } @Test @@ -148,9 +150,8 @@ public void getColumnCapabilitiesForLongColumn() Assert.assertEquals(ValueType.LONG, capabilities.getType()); Assert.assertFalse(capabilities.isDictionaryEncoded()); Assert.assertFalse(capabilities.hasBitmapIndexes()); - Assert.assertFalse(capabilities.hasMultipleValues()); + Assert.assertFalse(capabilities.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(capabilities.hasSpatialIndexes()); - Assert.assertTrue(capabilities.isComplete()); } @Test @@ -190,9 +191,9 @@ public void makeJoinMatcherWithDimensionSelectorOnString() } @Test - public void getCorrelatedColummnValuesMissingSearchColumnShouldReturnEmptySet() + public void getCorrelatedColummnValuesMissingSearchColumnShouldReturnEmpty() { - Set correlatedValues = + Optional> correlatedValues = target.getCorrelatedColumnValues( UNKNOWN_COLUMN, "foo", @@ -200,13 +201,13 @@ public void getCorrelatedColummnValuesMissingSearchColumnShouldReturnEmptySet() MAX_CORRELATION_SET_SIZE, false); - Assert.assertEquals(Collections.emptySet(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); } @Test - public void getCorrelatedColummnValuesMissingRetrievalColumnShouldReturnEmptySet() + public void getCorrelatedColummnValuesMissingRetrievalColumnShouldReturnEmpty() { - Set correlatedValues = + Optional> correlatedValues = target.getCorrelatedColumnValues( KEY_COLUMN, "foo", @@ -214,121 +215,121 @@ public void getCorrelatedColummnValuesMissingRetrievalColumnShouldReturnEmptySet MAX_CORRELATION_SET_SIZE, false); - Assert.assertEquals(Collections.emptySet(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchKeyAndRetrieveKeyColumnShouldReturnSearchValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( KEY_COLUMN, SEARCH_KEY_VALUE, KEY_COLUMN, MAX_CORRELATION_SET_SIZE, false); - Assert.assertEquals(ImmutableSet.of(SEARCH_KEY_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_KEY_VALUE)), correlatedValues); } @Test - public void getCorrelatedColumnValuesForSearchKeyAndRetrieveKeyColumnAboveLimitShouldReturnEmptySet() + public void getCorrelatedColumnValuesForSearchKeyAndRetrieveKeyColumnAboveLimitShouldReturnEmpty() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( KEY_COLUMN, SEARCH_KEY_VALUE, KEY_COLUMN, 0, false); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchKeyAndRetrieveValueColumnShouldReturnExtractedValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( KEY_COLUMN, SEARCH_KEY_VALUE, VALUE_COLUMN, MAX_CORRELATION_SET_SIZE, false); - Assert.assertEquals(ImmutableSet.of(SEARCH_VALUE_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_VALUE_VALUE)), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchKeyMissingAndRetrieveValueColumnShouldReturnExtractedValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( KEY_COLUMN, SEARCH_KEY_NULL_VALUE, VALUE_COLUMN, MAX_CORRELATION_SET_SIZE, false); - Assert.assertEquals(Collections.singleton(null), correlatedValues); + Assert.assertEquals(Optional.of(Collections.singleton(null)), correlatedValues); } @Test - public void getCorrelatedColumnValuesForSearchValueAndRetrieveValueColumnAndNonKeyColumnSearchDisabledShouldReturnSearchValue() + public void getCorrelatedColumnValuesForSearchValueAndRetrieveValueColumnAndNonKeyColumnSearchDisabledShouldReturnEmpty() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( VALUE_COLUMN, SEARCH_VALUE_VALUE, VALUE_COLUMN, MAX_CORRELATION_SET_SIZE, false); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); correlatedValues = target.getCorrelatedColumnValues( VALUE_COLUMN, SEARCH_VALUE_VALUE, KEY_COLUMN, 10, false); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchValueAndRetrieveValueColumnShouldReturnSearchValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( VALUE_COLUMN, SEARCH_VALUE_VALUE, VALUE_COLUMN, MAX_CORRELATION_SET_SIZE, true); - Assert.assertEquals(ImmutableSet.of(SEARCH_VALUE_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_VALUE_VALUE)), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchValueAndRetrieveKeyColumnShouldReturnUnAppliedValue() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( VALUE_COLUMN, SEARCH_VALUE_VALUE, KEY_COLUMN, 10, true); - Assert.assertEquals(ImmutableSet.of(SEARCH_KEY_VALUE), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of(SEARCH_KEY_VALUE)), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchValueAndRetrieveKeyColumnWithMaxLimitSetShouldHonorMaxLimit() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( VALUE_COLUMN, SEARCH_VALUE_VALUE, KEY_COLUMN, 0, true); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.empty(), correlatedValues); } @Test public void getCorrelatedColumnValuesForSearchUnknownValueAndRetrieveKeyColumnShouldReturnNoCorrelatedValues() { - Set correlatedValues = target.getCorrelatedColumnValues( + Optional> correlatedValues = target.getCorrelatedColumnValues( VALUE_COLUMN, SEARCH_VALUE_UNKNOWN, KEY_COLUMN, 10, true); - Assert.assertEquals(ImmutableSet.of(), correlatedValues); + Assert.assertEquals(Optional.of(ImmutableSet.of()), correlatedValues); } } diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/LookupJoinMatcherTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/LookupJoinMatcherTest.java index f5f7f3ff0d17..08a1a452af9c 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/LookupJoinMatcherTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/LookupJoinMatcherTest.java @@ -22,11 +22,13 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.data.ArrayBasedIndexedInts; import org.apache.druid.segment.data.SingleIndexedInt; import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.lookup.LookupJoinMatcher; @@ -150,6 +152,40 @@ public void testMatchConditionSometimesTrueSometimesFalse() Assert.assertFalse(target.hasMatch()); } + @Test(expected = QueryUnsupportedException.class) + public void testMatchMultiValuedRowShouldThrowException() + { + ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{2, 4, 6}); + Mockito.doReturn(dimensionSelector).when(leftSelectorFactory).makeDimensionSelector(ArgumentMatchers.any(DimensionSpec.class)); + Mockito.doReturn(row).when(dimensionSelector).getRow(); + + JoinConditionAnalysis condition = JoinConditionAnalysis.forExpression( + StringUtils.format("\"%sk\" == foo", PREFIX), + PREFIX, + ExprMacroTable.nil() + ); + target = LookupJoinMatcher.create(extractor, leftSelectorFactory, condition, true); + // Test match should throw exception + target.matchCondition(); + } + + @Test + public void testMatchEmptyRow() + { + ArrayBasedIndexedInts row = new ArrayBasedIndexedInts(new int[]{}); + Mockito.doReturn(dimensionSelector).when(leftSelectorFactory).makeDimensionSelector(ArgumentMatchers.any(DimensionSpec.class)); + Mockito.doReturn(row).when(dimensionSelector).getRow(); + + JoinConditionAnalysis condition = JoinConditionAnalysis.forExpression( + StringUtils.format("\"%sk\" == foo", PREFIX), + PREFIX, + ExprMacroTable.nil() + ); + target = LookupJoinMatcher.create(extractor, leftSelectorFactory, condition, true); + target.matchCondition(); + Assert.assertFalse(target.hasMatch()); + } + private void verifyMatch(String expectedKey, String expectedValue) { DimensionSelector selector = target.getColumnSelectorFactory() diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java index 49ebc36465b8..31bdc56a0ec7 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/RowBasedIndexedTableTest.java @@ -172,4 +172,11 @@ public void test_columnReader_countriesOutOfBoundsColumn() expectedException.expect(IndexOutOfBoundsException.class); countriesTable.columnReader(99); } + + @Test + public void testVersion() + { + Assert.assertEquals(JoinTestHelper.INDEXED_TABLE_VERSION, countriesTable.version()); + Assert.assertEquals(JoinTestHelper.INDEXED_TABLE_VERSION, regionsTable.version()); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java index d17200fb5b0e..c16946093723 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumnTest.java @@ -79,12 +79,12 @@ public DummyStringVirtualColumnTest() mixedSegments = Lists.newArrayList(incrementalIndexSegment, queryableIndexSegment); topNTestHelper = AggregationTestHelper.createTopNQueryAggregationTestHelper( - Collections.EMPTY_LIST, + Collections.emptyList(), null ); groupByTestHelper = AggregationTestHelper.createGroupByQueryAggregationTestHelper( - Collections.EMPTY_LIST, + Collections.emptyList(), new GroupByQueryConfig(), null ); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java index 526c63c60c17..16e090dc3d53 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -357,8 +357,7 @@ public ColumnCapabilities getColumnCapabilities(String column) { return new ColumnCapabilitiesImpl().setType(ValueType.STRING) .setHasMultipleValues(true) - .setDictionaryEncoded(true) - .setIsComplete(true); + .setDictionaryEncoded(true); } }; final BaseObjectColumnValueSelector selectorImplicit = @@ -814,9 +813,9 @@ public void testCapabilities() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertTrue(caps.hasMultipleValues()); + Assert.assertTrue(caps.hasMultipleValues().isUnknown()); + Assert.assertTrue(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertFalse(caps.isComplete()); caps = Z_CONCAT_X.capabilities("expr"); Assert.assertEquals(ValueType.STRING, caps.getType()); @@ -824,8 +823,8 @@ public void testCapabilities() Assert.assertFalse(caps.isDictionaryEncoded()); Assert.assertFalse(caps.areDictionaryValuesSorted().isTrue()); Assert.assertFalse(caps.areDictionaryValuesUnique().isTrue()); - Assert.assertTrue(caps.hasMultipleValues()); + Assert.assertTrue(caps.hasMultipleValues().isUnknown()); + Assert.assertTrue(caps.hasMultipleValues().isMaybeTrue()); Assert.assertFalse(caps.hasSpatialIndexes()); - Assert.assertFalse(caps.isComplete()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java index b1a9eeeab14d..597dbb6ee4e4 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java @@ -417,7 +417,7 @@ public boolean isNull() @Override public ColumnCapabilities capabilities(String columnName) { - return new ColumnCapabilitiesImpl().setType(ValueType.LONG); + return ColumnCapabilitiesImpl.createSimpleNumericColumnCapabilities(ValueType.LONG); } @Override diff --git a/server/pom.xml b/server/pom.xml index 5d68a91dfd89..ddce092c1cd4 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -291,10 +291,6 @@ javax.validation validation-api - - org.hibernate - hibernate-validator - com.google.errorprone error_prone_annotations diff --git a/server/src/main/java/org/apache/druid/client/BrokerServerView.java b/server/src/main/java/org/apache/druid/client/BrokerServerView.java index 337e9a46fb2f..debd72131c23 100644 --- a/server/src/main/java/org/apache/druid/client/BrokerServerView.java +++ b/server/src/main/java/org/apache/druid/client/BrokerServerView.java @@ -42,6 +42,7 @@ import org.apache.druid.query.TableDataSource; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -219,39 +220,52 @@ private void serverAddedSegment(final DruidServerMetadata server, final DataSegm { SegmentId segmentId = segment.getId(); synchronized (lock) { - log.debug("Adding segment[%s] for server[%s]", segment, server); - - ServerSelector selector = selectors.get(segmentId); - if (selector == null) { - selector = new ServerSelector(segment, tierSelectorStrategy); + // in theory we could probably just filter this to ensure we don't put ourselves in here, to make broker tree + // query topologies, but for now just skip all brokers, so we don't create some sort of wild infinite query + // loop... + if (!server.getType().equals(ServerType.BROKER)) { + log.debug("Adding segment[%s] for server[%s]", segment, server); + ServerSelector selector = selectors.get(segmentId); + if (selector == null) { + selector = new ServerSelector(segment, tierSelectorStrategy); + + VersionedIntervalTimeline timeline = timelines.get(segment.getDataSource()); + if (timeline == null) { + timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + timelines.put(segment.getDataSource(), timeline); + } - VersionedIntervalTimeline timeline = timelines.get(segment.getDataSource()); - if (timeline == null) { - timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - timelines.put(segment.getDataSource(), timeline); + timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); + selectors.put(segmentId, selector); } - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector)); - selectors.put(segmentId, selector); - } - - QueryableDruidServer queryableDruidServer = clients.get(server.getName()); - if (queryableDruidServer == null) { - queryableDruidServer = addServer(baseView.getInventoryValue(server.getName())); + QueryableDruidServer queryableDruidServer = clients.get(server.getName()); + if (queryableDruidServer == null) { + queryableDruidServer = addServer(baseView.getInventoryValue(server.getName())); + } + selector.addServerAndUpdateSegment(queryableDruidServer, segment); } - selector.addServerAndUpdateSegment(queryableDruidServer, segment); + // run the callbacks, even if the segment came from a broker, lets downstream watchers decide what to do with it runTimelineCallbacks(callback -> callback.segmentAdded(server, segment)); } } private void serverRemovedSegment(DruidServerMetadata server, DataSegment segment) { + SegmentId segmentId = segment.getId(); final ServerSelector selector; synchronized (lock) { log.debug("Removing segment[%s] from server[%s].", segmentId, server); + // we don't store broker segments here, but still run the callbacks for the segment being removed from the server + // since the broker segments are not stored on the timeline, do not fire segmentRemoved event + if (server.getType().equals(ServerType.BROKER)) { + runTimelineCallbacks(callback -> callback.serverSegmentRemoved(server, segment)); + return; + } + selector = selectors.get(segmentId); if (selector == null) { log.warn("Told to remove non-existant segment[%s]", segmentId); diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index ae4dc1689ccf..ded9cf09d4ee 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -261,7 +261,7 @@ private class SpecificQueryRunnable // For nested queries, we need to look at the intervals of the inner most query. this.intervals = dataSourceAnalysis.getBaseQuerySegmentSpec() .map(QuerySegmentSpec::getIntervals) - .orElse(query.getIntervals()); + .orElseGet(() -> query.getIntervals()); } private ImmutableMap makeDownstreamQueryContext() @@ -452,7 +452,7 @@ private String computeCurrentEtag(final Set segments, @Nu Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; for (SegmentServerSelector p : segments) { - if (!p.getServer().pick().getServer().segmentReplicatable()) { + if (!p.getServer().pick().getServer().isSegmentReplicationTarget()) { hasOnlyHistoricalSegments = false; break; } @@ -633,7 +633,7 @@ private void addSequencesFromServer( if (isBySegment) { serverResults = getBySegmentServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); - } else if (!server.segmentReplicatable() || !populateCache) { + } else if (!server.isSegmentReplicationTarget() || !populateCache) { serverResults = getSimpleServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); } else { serverResults = getAndCacheServerResults(serverRunner, segmentsOfServer, maxQueuedBytesPerServer); diff --git a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java index 2517a8f0e9be..538cc2f526f3 100644 --- a/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java +++ b/server/src/main/java/org/apache/druid/client/CoordinatorServerView.java @@ -200,6 +200,10 @@ public VersionedIntervalTimeline getTimeline(DataSource } } + public Map getSegmentLoadInfos() + { + return segmentLoadInfos; + } @Override public DruidServer getInventoryValue(String serverKey) diff --git a/server/src/main/java/org/apache/druid/client/DruidServer.java b/server/src/main/java/org/apache/druid/client/DruidServer.java index ddcba54f1c3c..6c52866d0586 100644 --- a/server/src/main/java/org/apache/druid/client/DruidServer.java +++ b/server/src/main/java/org/apache/druid/client/DruidServer.java @@ -137,9 +137,19 @@ public String getTier() return metadata.getTier(); } - public boolean segmentReplicatable() + public boolean isSegmentReplicationTarget() { - return metadata.segmentReplicatable(); + return metadata.isSegmentReplicationTarget(); + } + + public boolean isSegmentBroadcastTarget() + { + return metadata.isSegmentBroadcastTarget(); + } + + public boolean isSegmentReplicationOrBroadcastTarget() + { + return metadata.isSegmentReplicationTarget() || metadata.isSegmentBroadcastTarget(); } @JsonProperty diff --git a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java index 733202981839..898ce7c75956 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java @@ -44,7 +44,8 @@ public abstract class DruidNodeDiscoveryProvider private static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.BROKER, NodeRole.HISTORICAL, NodeRole.PEON, NodeRole.INDEXER), - DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.HISTORICAL, NodeRole.PEON, NodeRole.INDEXER), + DataNodeService.DISCOVERY_SERVICE_KEY, + ImmutableSet.of(NodeRole.HISTORICAL, NodeRole.PEON, NodeRole.INDEXER, NodeRole.BROKER), WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.MIDDLE_MANAGER, NodeRole.INDEXER) ); diff --git a/server/src/main/java/org/apache/druid/guice/ExpressionModule.java b/server/src/main/java/org/apache/druid/guice/ExpressionModule.java index f695563d352f..9e451e8f12f9 100644 --- a/server/src/main/java/org/apache/druid/guice/ExpressionModule.java +++ b/server/src/main/java/org/apache/druid/guice/ExpressionModule.java @@ -31,6 +31,7 @@ import org.apache.druid.query.expression.IPv4AddressStringifyExprMacro; import org.apache.druid.query.expression.LikeExprMacro; import org.apache.druid.query.expression.RegexpExtractExprMacro; +import org.apache.druid.query.expression.RegexpLikeExprMacro; import org.apache.druid.query.expression.TimestampCeilExprMacro; import org.apache.druid.query.expression.TimestampExtractExprMacro; import org.apache.druid.query.expression.TimestampFloorExprMacro; @@ -41,8 +42,6 @@ import java.util.List; -/** - */ public class ExpressionModule implements DruidModule { public static final List> EXPR_MACROS = @@ -52,6 +51,7 @@ public class ExpressionModule implements DruidModule .add(IPv4AddressStringifyExprMacro.class) .add(LikeExprMacro.class) .add(RegexpExtractExprMacro.class) + .add(RegexpLikeExprMacro.class) .add(TimestampCeilExprMacro.class) .add(TimestampExtractExprMacro.class) .add(TimestampFloorExprMacro.class) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java index eb9e9916cac8..e6fdcfb45f1a 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataStorageActionHandler.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.Maps; @@ -173,8 +172,7 @@ public void insert( } } - @VisibleForTesting - protected static boolean isStatementException(Throwable e) + public static boolean isStatementException(Throwable e) { return e instanceof StatementException || (e instanceof CallbackFailedException && e.getCause() instanceof StatementException); diff --git a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java index 4f97b158021e..889141a89c15 100644 --- a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataManager.java @@ -20,6 +20,7 @@ package org.apache.druid.metadata; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.timeline.DataSegment; @@ -113,6 +114,22 @@ int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) */ Iterable iterateAllUsedSegments(); + /** + * Returns an iterable to go over all used and non-overshadowed segments of given data sources over given interval. + * The order in which segments are iterated is unspecified. Note: the iteration may not be as trivially cheap as, + * for example, iteration over an ArrayList. Try (to some reasonable extent) to organize the code so that it + * iterates the returned iterable only once rather than several times. + * If {@param requiresLatest} is true then a force metadatastore poll will be triggered. This can cause a longer + * response time but will ensure that the latest segment information (at the time this method is called) is returned. + * If {@param requiresLatest} is false then segment information from stale snapshot of up to the last periodic poll + * period {@link SqlSegmentsMetadataManager#periodicPollDelay} will be used. + */ + Optional> iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval( + String datasource, + Interval interval, + boolean requiresLatest + ); + /** * Retrieves all data source names for which there are segment in the database, regardless of whether those segments * are used or not. If there are no segments in the database, returns an empty set. diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java index 92a874863527..60f7f4b86eeb 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataManager.java @@ -20,6 +20,8 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; import com.google.common.base.Throwables; @@ -44,6 +46,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @@ -95,7 +98,8 @@ private interface DatabasePoll {} /** Represents periodic {@link #poll}s happening from {@link #exec}. */ - private static class PeriodicDatabasePoll implements DatabasePoll + @VisibleForTesting + static class PeriodicDatabasePoll implements DatabasePoll { /** * This future allows to wait until {@link #dataSourcesSnapshot} is initialized in the first {@link #poll()} @@ -104,13 +108,15 @@ private static class PeriodicDatabasePoll implements DatabasePoll * leadership changes. */ final CompletableFuture firstPollCompletionFuture = new CompletableFuture<>(); + long lastPollStartTimestampInMs = -1; } /** * Represents on-demand {@link #poll} initiated at periods of time when SqlSegmentsMetadataManager doesn't poll the database * periodically. */ - private static class OnDemandDatabasePoll implements DatabasePoll + @VisibleForTesting + static class OnDemandDatabasePoll implements DatabasePoll { final long initiationTimeNanos = System.nanoTime(); final CompletableFuture pollCompletionFuture = new CompletableFuture<>(); @@ -127,7 +133,7 @@ long nanosElapsedFromInitiation() * called at the same time if two different threads are calling them. This might be possible if Coordinator gets and * drops leadership repeatedly in quick succession. * - * This lock is also used to synchronize {@link #awaitOrPerformDatabasePoll} for times when SqlSegmentsMetadataManager + * This lock is also used to synchronize {@link #useLatestIfWithinDelayOrPerformNewDatabasePoll} for times when SqlSegmentsMetadataManager * is not polling the database periodically (in other words, when the Coordinator is not the leader). */ private final ReentrantReadWriteLock startStopPollLock = new ReentrantReadWriteLock(); @@ -155,7 +161,7 @@ long nanosElapsedFromInitiation() * easy to forget to do. * * This field may be updated from {@link #exec}, or from whatever thread calling {@link #doOnDemandPoll} via {@link - * #awaitOrPerformDatabasePoll()} via one of the public methods of SqlSegmentsMetadataManager. + * #useLatestIfWithinDelayOrPerformNewDatabasePoll()} via one of the public methods of SqlSegmentsMetadataManager. */ private volatile @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = null; @@ -170,7 +176,7 @@ long nanosElapsedFromInitiation() * Note that if there is a happens-before relationship between a call to {@link #startPollingDatabasePeriodically()} * (on Coordinators' leadership change) and one of the methods accessing the {@link #dataSourcesSnapshot}'s state in * this class the latter is guaranteed to await for the initiated periodic poll. This is because when the latter - * method calls to {@link #awaitLatestDatabasePoll()} via {@link #awaitOrPerformDatabasePoll}, they will + * method calls to {@link #useLatestSnapshotIfWithinDelay()} via {@link #useLatestIfWithinDelayOrPerformNewDatabasePoll}, they will * see the latest {@link PeriodicDatabasePoll} value (stored in this field, latestDatabasePoll, in {@link * #startPollingDatabasePeriodically()}) and to await on its {@link PeriodicDatabasePoll#firstPollCompletionFuture}. * @@ -185,7 +191,7 @@ long nanosElapsedFromInitiation() * SegmentsMetadataManager} and guarantee that it always returns consistent and relatively up-to-date data from methods * like {@link #getImmutableDataSourceWithUsedSegments}, while avoiding excessive repetitive polls. The last part * is achieved via "hooking on" other polls by awaiting on {@link PeriodicDatabasePoll#firstPollCompletionFuture} or - * {@link OnDemandDatabasePoll#pollCompletionFuture}, see {@link #awaitOrPerformDatabasePoll} method + * {@link OnDemandDatabasePoll#pollCompletionFuture}, see {@link #useLatestIfWithinDelayOrPerformNewDatabasePoll} method * implementation for details. * * Note: the overall implementation of periodic/on-demand polls is not completely optimal: for example, when the @@ -194,7 +200,7 @@ long nanosElapsedFromInitiation() * during Coordinator leadership switches is not a priority. * * This field is {@code volatile} because it's checked and updated in a double-checked locking manner in {@link - * #awaitOrPerformDatabasePoll()}. + * #useLatestIfWithinDelayOrPerformNewDatabasePoll()}. */ private volatile @Nullable DatabasePoll latestDatabasePoll = null; @@ -311,6 +317,22 @@ public void startPollingDatabasePeriodically() private Runnable createPollTaskForStartOrder(long startOrder, PeriodicDatabasePoll periodicDatabasePoll) { return () -> { + // If latest poll was an OnDemandDatabasePoll that started less than periodicPollDelay, + // We will wait for (periodicPollDelay - currentTime - LatestOnDemandDatabasePollStartTime) then check again. + try { + long periodicPollDelayNanos = TimeUnit.MILLISECONDS.toNanos(periodicPollDelay.getMillis()); + while (latestDatabasePoll != null + && latestDatabasePoll instanceof OnDemandDatabasePoll + && ((OnDemandDatabasePoll) latestDatabasePoll).nanosElapsedFromInitiation() < periodicPollDelayNanos) { + long sleepNano = periodicPollDelayNanos + - ((OnDemandDatabasePoll) latestDatabasePoll).nanosElapsedFromInitiation(); + TimeUnit.NANOSECONDS.sleep(sleepNano); + } + } + catch (Exception e) { + log.debug(e, "Exception found while waiting for next periodic poll"); + } + // poll() is synchronized together with startPollingDatabasePeriodically(), stopPollingDatabasePeriodically() and // isPollingDatabasePeriodically() to ensure that when stopPollingDatabasePeriodically() exits, poll() won't // actually run anymore after that (it could only enter the synchronized section and exit immediately because the @@ -320,8 +342,10 @@ private Runnable createPollTaskForStartOrder(long startOrder, PeriodicDatabasePo lock.lock(); try { if (startOrder == currentStartPollingOrder) { + periodicDatabasePoll.lastPollStartTimestampInMs = System.currentTimeMillis(); poll(); periodicDatabasePoll.firstPollCompletionFuture.complete(null); + latestDatabasePoll = periodicDatabasePoll; } else { log.debug("startOrder = currentStartPollingOrder = %d, skipping poll()", startOrder); } @@ -381,16 +405,16 @@ public void stopPollingDatabasePeriodically() } } - private void awaitOrPerformDatabasePoll() + private void useLatestIfWithinDelayOrPerformNewDatabasePoll() { - // Double-checked locking with awaitLatestDatabasePoll() call playing the role of the "check". - if (awaitLatestDatabasePoll()) { + // Double-checked locking with useLatestSnapshotIfWithinDelay() call playing the role of the "check". + if (useLatestSnapshotIfWithinDelay()) { return; } ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); lock.lock(); try { - if (awaitLatestDatabasePoll()) { + if (useLatestSnapshotIfWithinDelay()) { return; } OnDemandDatabasePoll onDemandDatabasePoll = new OnDemandDatabasePoll(); @@ -403,11 +427,17 @@ private void awaitOrPerformDatabasePoll() } /** - * If the latest {@link DatabasePoll} is a {@link PeriodicDatabasePoll}, or an {@link OnDemandDatabasePoll} that is - * made not longer than {@link #periodicPollDelay} from now, awaits for it and returns true; returns false otherwise, - * meaning that a new on-demand database poll should be initiated. + * This method returns true without waiting for database poll if the latest {@link DatabasePoll} is a + * {@link PeriodicDatabasePoll} that has completed it's first poll, or an {@link OnDemandDatabasePoll} that is + * made not longer than {@link #periodicPollDelay} from current time. + * This method does wait untill completion for if the latest {@link DatabasePoll} is a + * {@link PeriodicDatabasePoll} that has not completed it's first poll, or an {@link OnDemandDatabasePoll} that is + * already in the process of polling the database. + * This means that any method using this check can read from snapshot that is + * up to {@link SqlSegmentsMetadataManager#periodicPollDelay} old. */ - private boolean awaitLatestDatabasePoll() + @VisibleForTesting + boolean useLatestSnapshotIfWithinDelay() { DatabasePoll latestDatabasePoll = this.latestDatabasePoll; if (latestDatabasePoll instanceof PeriodicDatabasePoll) { @@ -430,6 +460,49 @@ private boolean awaitLatestDatabasePoll() return false; } + /** + * This method will always force a database poll if there is no ongoing database poll. This method will then + * waits for the new poll or the ongoing poll to completes before returning. + * This means that any method using this check can be sure that the latest poll for the snapshot was completed after + * this method was called. + */ + @VisibleForTesting + void forceOrWaitOngoingDatabasePoll() + { + long checkStartTime = System.currentTimeMillis(); + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); + lock.lock(); + try { + DatabasePoll latestDatabasePoll = this.latestDatabasePoll; + try { + //Verify if there was a periodic poll completed while we were waiting for the lock + if (latestDatabasePoll instanceof PeriodicDatabasePoll + && ((PeriodicDatabasePoll) latestDatabasePoll).lastPollStartTimestampInMs > checkStartTime) { + return; + } + // Verify if there was a on-demand poll completed while we were waiting for the lock + if (latestDatabasePoll instanceof OnDemandDatabasePoll) { + long checkStartTimeNanos = TimeUnit.MILLISECONDS.toNanos(checkStartTime); + OnDemandDatabasePoll latestOnDemandPoll = (OnDemandDatabasePoll) latestDatabasePoll; + if (latestOnDemandPoll.initiationTimeNanos > checkStartTimeNanos) { + return; + } + } + } + catch (Exception e) { + // Latest poll was unsuccessful, try to do a new poll + log.debug(e, "Latest poll was unsuccessful. Starting a new poll..."); + } + // Force a database poll + OnDemandDatabasePoll onDemandDatabasePoll = new OnDemandDatabasePoll(); + this.latestDatabasePoll = onDemandDatabasePoll; + doOnDemandPoll(onDemandDatabasePoll); + } + finally { + lock.unlock(); + } + } + private void doOnDemandPoll(OnDemandDatabasePoll onDemandPoll) { try { @@ -857,19 +930,44 @@ public Set getOvershadowedSegments() @Override public DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments() { - awaitOrPerformDatabasePoll(); + useLatestIfWithinDelayOrPerformNewDatabasePoll(); return dataSourcesSnapshot; } + @VisibleForTesting + DataSourcesSnapshot getDataSourcesSnapshot() + { + return dataSourcesSnapshot; + } + + @VisibleForTesting + DatabasePoll getLatestDatabasePoll() + { + return latestDatabasePoll; + } + + @Override public Iterable iterateAllUsedSegments() { - awaitOrPerformDatabasePoll(); - return () -> dataSourcesSnapshot - .getDataSourcesWithAllUsedSegments() - .stream() - .flatMap(dataSource -> dataSource.getSegments().stream()) - .iterator(); + useLatestIfWithinDelayOrPerformNewDatabasePoll(); + return dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot(); + } + + @Override + public Optional> iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(String datasource, + Interval interval, + boolean requiresLatest) + { + if (requiresLatest) { + forceOrWaitOngoingDatabasePoll(); + } else { + useLatestIfWithinDelayOrPerformNewDatabasePoll(); + } + VersionedIntervalTimeline usedSegmentsTimeline + = dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource().get(datasource); + return Optional.fromNullable(usedSegmentsTimeline) + .transform(timeline -> timeline.findNonOvershadowedObjectsInInterval(interval, Partitions.ONLY_COMPLETE)); } @Override diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java b/server/src/main/java/org/apache/druid/metadata/input/InputSourceModule.java similarity index 51% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java rename to server/src/main/java/org/apache/druid/metadata/input/InputSourceModule.java index 85574b7b254b..0423af4f9a03 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/GeneratedHashPartitionsReport.java +++ b/server/src/main/java/org/apache/druid/metadata/input/InputSourceModule.java @@ -17,28 +17,35 @@ * under the License. */ -package org.apache.druid.indexing.common.task.batch.parallel; +package org.apache.druid.metadata.input; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.google.common.collect.ImmutableList; +import com.google.inject.Binder; +import org.apache.druid.initialization.DruidModule; import java.util.List; /** - * Report containing the {@link HashPartitionStat}s created by a {@link PartialHashSegmentGenerateTask}. - * This report is collected by {@link ParallelIndexSupervisorTask} and - * used to generate {@link PartialHashSegmentMergeIOConfig}. + * Module that installs {@link org.apache.druid.data.input.InputSource} implementations */ -class GeneratedHashPartitionsReport extends GeneratedPartitionsReport implements SubTaskReport +public class InputSourceModule implements DruidModule { - public static final String TYPE = "generated_partitions"; + @Override + public List getJacksonModules() + { + return ImmutableList.of( + new SimpleModule("InputSourceModule") + .registerSubtypes( + new NamedType(SqlInputSource.class, "sql") + ) + ); + } - @JsonCreator - GeneratedHashPartitionsReport( - @JsonProperty("taskId") String taskId, - @JsonProperty("partitionStats") List partitionStats - ) + @Override + public void configure(Binder binder) { - super(taskId, partitionStats); } } diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java b/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java new file mode 100644 index 000000000000..724077a4c0d2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlEntity.java @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata.input; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.SQLMetadataStorageActionHandler; +import org.skife.jdbi.v2.ResultIterator; +import org.skife.jdbi.v2.exceptions.ResultSetException; + +import javax.annotation.Nullable; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.HashMap; +import java.util.Map; + +/** + * Represents a rdbms based input resource and knows how to read query results from the resource using SQL queries. + */ +public class SqlEntity implements InputEntity +{ + private static final Logger LOG = new Logger(SqlEntity.class); + + private final String sql; + private final ObjectMapper objectMapper; + private final SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector; + private final boolean foldCase; + + public SqlEntity( + String sql, + SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + boolean foldCase, + ObjectMapper objectMapper + ) + { + this.sql = sql; + this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( + sqlFirehoseDatabaseConnector, + "SQL Metadata Connector not configured!" + ); + this.foldCase = foldCase; + this.objectMapper = objectMapper; + } + + public String getSql() + { + return sql; + } + + @Nullable + @Override + public URI getUri() + { + return null; + } + + @Override + public InputStream open() + { + throw new UnsupportedOperationException("Please use fetch() instead"); + } + + @Override + public CleanableFile fetch(File temporaryDirectory, byte[] fetchBuffer) throws IOException + { + final File tempFile = File.createTempFile("druid-sql-entity", ".tmp", temporaryDirectory); + return openCleanableFile(sql, sqlFirehoseDatabaseConnector, objectMapper, foldCase, tempFile); + + } + + /** + * Executes a SQL query on the specified database and fetches the result into the given file. + * The result file is deleted if the query execution or the file write fails. + * + * @param sql The SQL query to be executed + * @param sqlFirehoseDatabaseConnector The database connector + * @param objectMapper An object mapper, used for deserialization + * @param foldCase A boolean flag used to enable or disabling case sensitivity while handling database column names + * + * @return A {@link InputEntity.CleanableFile} object that wraps the file containing the SQL results + */ + + public static CleanableFile openCleanableFile( + String sql, + SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + ObjectMapper objectMapper, + boolean foldCase, + File tempFile + ) + throws IOException + { + try (FileOutputStream fos = new FileOutputStream(tempFile); + final JsonGenerator jg = objectMapper.getFactory().createGenerator(fos);) { + + // Execute the sql query and lazily retrieve the results into the file in json format. + // foldCase is useful to handle differences in case sensitivity behavior across databases. + sqlFirehoseDatabaseConnector.retryWithHandle( + (handle) -> { + ResultIterator> resultIterator = handle.createQuery( + sql + ).map( + (index, r, ctx) -> { + Map resultRow = foldCase ? new CaseFoldedMap() : new HashMap<>(); + ResultSetMetaData resultMetadata; + try { + resultMetadata = r.getMetaData(); + } + catch (SQLException e) { + throw new ResultSetException("Unable to obtain metadata from result set", e, ctx); + } + try { + for (int i = 1; i <= resultMetadata.getColumnCount(); i++) { + String key = resultMetadata.getColumnName(i); + String alias = resultMetadata.getColumnLabel(i); + Object value = r.getObject(i); + resultRow.put(alias != null ? alias : key, value); + } + } + catch (SQLException e) { + throw new ResultSetException("Unable to access specific metadata from " + + "result set metadata", e, ctx); + } + return resultRow; + } + ).iterator(); + jg.writeStartArray(); + while (resultIterator.hasNext()) { + jg.writeObject(resultIterator.next()); + } + jg.writeEndArray(); + jg.close(); + return null; + }, + (exception) -> sqlFirehoseDatabaseConnector.isTransientException(exception) + && !(SQLMetadataStorageActionHandler.isStatementException(exception)) + ); + return new CleanableFile() + { + @Override + public File file() + { + return tempFile; + } + + @Override + public void close() + { + if (!tempFile.delete()) { + LOG.warn("Failed to remove file[%s]", tempFile.getAbsolutePath()); + } + } + }; + } + catch (Exception e) { + if (!tempFile.delete()) { + LOG.warn("Failed to remove file[%s]", tempFile.getAbsolutePath()); + } + throw new IOException(e); + } + } + + private static class CaseFoldedMap extends HashMap + { + public static final long serialVersionUID = 1L; + + @Override + public Object get(Object obj) + { + return super.get(obj == null ? null : StringUtils.toLowerCase((String) obj)); + } + + @Override + public Object put(String key, Object value) + { + return super.put(key == null ? null : StringUtils.toLowerCase(key), value); + } + + @Override + public boolean containsKey(Object obj) + { + return super.containsKey(obj == null ? null : StringUtils.toLowerCase((String) obj)); + } + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlInputFormat.java b/server/src/main/java/org/apache/druid/metadata/input/SqlInputFormat.java new file mode 100644 index 000000000000..6d0aa59a20d2 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlInputFormat.java @@ -0,0 +1,54 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputEntityReader; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; + +import java.io.File; + +public class SqlInputFormat implements InputFormat +{ + private final ObjectMapper objectMapper; + + public SqlInputFormat(ObjectMapper objectMapper) + { + this.objectMapper = objectMapper; + } + + @Override + public boolean isSplittable() + { + return true; + } + + @Override + public InputEntityReader createReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory + ) + { + return new SqlReader(inputRowSchema, source, temporaryDirectory, objectMapper); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java b/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java new file mode 100644 index 000000000000..c7dfbb7fa365 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlInputSource.java @@ -0,0 +1,151 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import org.apache.druid.data.input.AbstractInputSource; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.SplitHintSpec; +import org.apache.druid.data.input.impl.InputEntityIteratingReader; +import org.apache.druid.data.input.impl.SplittableInputSource; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; + +import javax.annotation.Nullable; +import java.io.File; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.Stream; + +public class SqlInputSource extends AbstractInputSource implements SplittableInputSource +{ + private final List sqls; + private final SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector; + private final ObjectMapper objectMapper; + private final boolean foldCase; + + @JsonCreator + public SqlInputSource( + @JsonProperty("sqls") List sqls, + @JsonProperty("foldCase") boolean foldCase, + @JsonProperty("database") SQLFirehoseDatabaseConnector sqlFirehoseDatabaseConnector, + @JacksonInject @Smile ObjectMapper objectMapper + ) + { + Preconditions.checkArgument(sqls.size() > 0, "No SQL queries provided"); + + this.sqls = sqls; + this.foldCase = foldCase; + this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( + sqlFirehoseDatabaseConnector, + "SQL Metadata Connector not configured!" + ); + this.objectMapper = objectMapper; + } + + @JsonProperty + public List getSqls() + { + return sqls; + } + + @JsonProperty + public boolean isFoldCase() + { + return foldCase; + } + + @JsonProperty("database") + public SQLFirehoseDatabaseConnector getSQLFirehoseDatabaseConnector() + { + return sqlFirehoseDatabaseConnector; + } + + @Override + public Stream> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return sqls.stream().map(InputSplit::new); + } + + @Override + public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec) + { + return sqls.size(); + } + + @Override + public SplittableInputSource withSplit(InputSplit split) + { + return new SqlInputSource( + Collections.singletonList(split.get()), + foldCase, + sqlFirehoseDatabaseConnector, + objectMapper + ); + } + + @Override + protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory) + { + final SqlInputFormat inputFormat = new SqlInputFormat(objectMapper); + return new InputEntityIteratingReader( + inputRowSchema, + inputFormat, + createSplits(inputFormat, null) + .map(split -> new SqlEntity(split.get(), sqlFirehoseDatabaseConnector, foldCase, objectMapper)).iterator(), + temporaryDirectory + ); + } + + @Override + public boolean needsFormat() + { + return false; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + SqlInputSource that = (SqlInputSource) o; + return foldCase == that.foldCase && + sqls.equals(that.sqls) && + sqlFirehoseDatabaseConnector.equals(that.sqlFirehoseDatabaseConnector); + } + + @Override + public int hashCode() + { + return Objects.hash(sqls, sqlFirehoseDatabaseConnector, foldCase); + } +} diff --git a/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java b/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java new file mode 100644 index 000000000000..4657158c0463 --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/input/SqlReader.java @@ -0,0 +1,95 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.IntermediateRowParsingReader; +import org.apache.druid.data.input.impl.MapInputRowParser; +import org.apache.druid.data.input.impl.prefetch.JsonIterator; +import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.java.util.common.parsers.ParseException; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Reader exclusively for {@link SqlEntity} + */ +public class SqlReader extends IntermediateRowParsingReader> +{ + private final InputRowSchema inputRowSchema; + private final SqlEntity source; + private final File temporaryDirectory; + private final ObjectMapper objectMapper; + + + SqlReader( + InputRowSchema inputRowSchema, + InputEntity source, + File temporaryDirectory, + ObjectMapper objectMapper + ) + { + this.inputRowSchema = inputRowSchema; + this.source = (SqlEntity) source; + this.temporaryDirectory = temporaryDirectory; + this.objectMapper = objectMapper; + } + + @Override + protected CloseableIterator> intermediateRowIterator() throws IOException + { + final Closer closer = Closer.create(); + //The results are fetched into local storage as this avoids having to keep a persistent database connection for a long time + final InputEntity.CleanableFile resultFile = closer.register(source.fetch(temporaryDirectory, null)); + FileInputStream inputStream = new FileInputStream(resultFile.file()); + JsonIterator> jsonIterator = new JsonIterator<>(new TypeReference>() + { + }, inputStream, closer, objectMapper); + return jsonIterator; + } + + @Override + protected List parseInputRows(Map intermediateRow) throws ParseException + { + return Collections.singletonList( + MapInputRowParser.parse( + inputRowSchema.getTimestampSpec(), + inputRowSchema.getDimensionsSpec(), + intermediateRow + ) + ); + } + + @Override + protected Map toMap(Map intermediateRow) + { + return intermediateRow; + } +} diff --git a/server/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java b/server/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java index 3890e6c3736a..4eee53fde5dc 100644 --- a/server/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java +++ b/server/src/main/java/org/apache/druid/segment/join/InlineJoinableFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.segment.join; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.query.DataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.segment.join.table.IndexedTable; @@ -35,6 +36,15 @@ */ public class InlineJoinableFactory implements JoinableFactory { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + // this should always be true if this is access through MapJoinableFactory, but check just in case... + // further, this should not ever be legitimately called, because this method is used to avoid subquery joins + // which use the InlineJoinableFactory + return dataSource instanceof InlineDataSource; + } + @Override public Optional build(final DataSource dataSource, final JoinConditionAnalysis condition) { @@ -49,7 +59,8 @@ public Optional build(final DataSource dataSource, final JoinCondition inlineDataSource.getRowsAsList(), inlineDataSource.rowAdapter(), inlineDataSource.getRowSignature(), - rightKeyColumns + rightKeyColumns, + DateTimes.nowUtc().toString() ) ) ); diff --git a/server/src/main/java/org/apache/druid/segment/join/LookupJoinableFactory.java b/server/src/main/java/org/apache/druid/segment/join/LookupJoinableFactory.java index a6fd209b1d12..2dab0a616639 100644 --- a/server/src/main/java/org/apache/druid/segment/join/LookupJoinableFactory.java +++ b/server/src/main/java/org/apache/druid/segment/join/LookupJoinableFactory.java @@ -42,6 +42,13 @@ public LookupJoinableFactory(LookupExtractorFactoryContainerProvider lookupProvi this.lookupProvider = lookupProvider; } + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + // this should always be true if this is access through MapJoinableFactory, but check just in case... + return dataSource instanceof LookupDataSource; + } + @Override public Optional build(final DataSource dataSource, final JoinConditionAnalysis condition) { diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java index c6c57233738c..883ac5b0211e 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderConfig.java @@ -23,9 +23,9 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import org.apache.druid.utils.JvmUtils; -import org.hibernate.validator.constraints.NotEmpty; import java.io.File; +import java.util.Collections; import java.util.List; import java.util.concurrent.TimeUnit; @@ -34,8 +34,7 @@ public class SegmentLoaderConfig { @JsonProperty - @NotEmpty - private List locations = null; + private List locations = Collections.emptyList(); @JsonProperty("lazyLoadOnStart") private boolean lazyLoadOnStart = false; @@ -50,7 +49,7 @@ public class SegmentLoaderConfig private int announceIntervalMillis = 0; // do not background announce @JsonProperty("numLoadingThreads") - private int numLoadingThreads = JvmUtils.getRuntimeInfo().getAvailableProcessors(); + private int numLoadingThreads = Math.max(1, JvmUtils.getRuntimeInfo().getAvailableProcessors() / 6); @JsonProperty("numBootstrapThreads") private Integer numBootstrapThreads = null; diff --git a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java index 398ad679b484..b2ac7e8f35b0 100644 --- a/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java +++ b/server/src/main/java/org/apache/druid/segment/loading/SegmentLoaderLocalCacheManager.java @@ -89,7 +89,6 @@ public SegmentLoaderLocalCacheManager( this.indexIO = indexIO; this.config = config; this.jsonMapper = mapper; - this.locations = new ArrayList<>(); for (StorageLocationConfig locationConfig : config.getLocations()) { locations.add( diff --git a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java b/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.java deleted file mode 100644 index e874a30c86d9..000000000000 --- a/server/src/main/java/org/apache/druid/segment/realtime/CliIndexerDataSegmentServerAnnouncerLifecycleHandler.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.segment.realtime; - -import com.google.common.base.Throwables; -import com.google.inject.Inject; -import org.apache.druid.concurrent.LifecycleLock; -import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; - -import java.io.IOException; - -/** - * Ties the {@link DataSegmentServerAnnouncer} announce/unannounce to the lifecycle start and stop. - * - * Analogous to {@link org.apache.druid.server.coordination.SegmentLoadDropHandler} on the Historicals, - * but without segment cache management. - */ -@ManageLifecycle -public class CliIndexerDataSegmentServerAnnouncerLifecycleHandler -{ - private static final EmittingLogger LOG = new EmittingLogger(CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); - - private final DataSegmentServerAnnouncer dataSegmentServerAnnouncer; - - private final LifecycleLock lifecycleLock = new LifecycleLock(); - - @Inject - public CliIndexerDataSegmentServerAnnouncerLifecycleHandler( - DataSegmentServerAnnouncer dataSegmentServerAnnouncer - ) - { - this.dataSegmentServerAnnouncer = dataSegmentServerAnnouncer; - } - - @LifecycleStart - public void start() throws IOException - { - if (!lifecycleLock.canStart()) { - throw new RuntimeException("Lifecycle lock could not start"); - } - - try { - if (lifecycleLock.isStarted()) { - return; - } - - LOG.info("Starting..."); - try { - dataSegmentServerAnnouncer.announce(); - } - catch (Exception e) { - Throwables.propagateIfPossible(e, IOException.class); - throw new RuntimeException(e); - } - LOG.info("Started."); - lifecycleLock.started(); - } - finally { - lifecycleLock.exitStart(); - } - } - - @LifecycleStop - public void stop() - { - if (!lifecycleLock.canStop()) { - throw new RuntimeException("Lifecycle lock could not stop"); - } - - if (!lifecycleLock.isStarted()) { - return; - } - - LOG.info("Stopping..."); - try { - dataSegmentServerAnnouncer.unannounce(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - LOG.info("Stopped."); - } -} 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 fbd6e8d39885..bf5ba26c5299 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 @@ -19,18 +19,22 @@ package org.apache.druid.segment.realtime; +import com.google.common.annotations.VisibleForTesting; 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.ReferenceCountingSegment; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.Closeable; +import java.util.Optional; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; /** */ @@ -71,27 +75,6 @@ public Interval getSegmentDataInterval() return adapter.get().getDataInterval(); } - public ReferenceCountingSegment getIncrementedSegment() - { - ReferenceCountingSegment segment = adapter.get(); - while (true) { - if (segment.increment()) { - return segment; - } - // segment.increment() returned false, means it is closed. Since close() in swapSegment() happens after segment - // swap, the new segment should already be visible. - ReferenceCountingSegment newSegment = adapter.get(); - if (segment == newSegment) { - throw new ISE("segment.close() is called somewhere outside FireHydrant.swapSegment()"); - } - if (newSegment == null) { - throw new ISE("FireHydrant was 'closed' by swapping segment to null while acquiring a segment"); - } - segment = newSegment; - // Spin loop. - } - } - public int getCount() { return count; @@ -133,12 +116,75 @@ public void swapSegment(@Nullable Segment newSegment) } } - public Pair getAndIncrementSegment() + public ReferenceCountingSegment getIncrementedSegment() + { + ReferenceCountingSegment segment = adapter.get(); + while (true) { + if (segment.increment()) { + return segment; + } + // segment.increment() returned false, means it is closed. Since close() in swapSegment() happens after segment + // swap, the new segment should already be visible. + ReferenceCountingSegment newSegment = adapter.get(); + if (segment == newSegment) { + throw new ISE("segment.close() is called somewhere outside FireHydrant.swapSegment()"); + } + if (newSegment == null) { + throw new ISE("FireHydrant was 'closed' by swapping segment to null while acquiring a segment"); + } + segment = newSegment; + // Spin loop. + } + } + + public Pair getAndIncrementSegment() { ReferenceCountingSegment segment = getIncrementedSegment(); return new Pair<>(segment, segment.decrementOnceCloseable()); } + /** + * This method is like a combined form of {@link #getIncrementedSegment} and {@link #getAndIncrementSegment} that + * deals in {@link SegmentReference} instead of directly with {@link ReferenceCountingSegment} in order to acquire + * reference count for both hydrant's segment and any tracked joinables taking part in the query. + */ + public Optional> getSegmentForQuery( + Function segmentMapFn + ) + { + ReferenceCountingSegment sinkSegment = adapter.get(); + SegmentReference segment = segmentMapFn.apply(sinkSegment); + while (true) { + Optional reference = segment.acquireReferences(); + if (reference.isPresent()) { + + return Optional.of(new Pair<>(segment, reference.get())); + } + // segment.acquireReferences() returned false, means it is closed. Since close() in swapSegment() happens after + // segment swap, the new segment should already be visible. + ReferenceCountingSegment newSinkSegment = adapter.get(); + if (newSinkSegment == null) { + throw new ISE("FireHydrant was 'closed' by swapping segment to null while acquiring a segment"); + } + if (sinkSegment == newSinkSegment) { + if (newSinkSegment.isClosed()) { + throw new ISE("segment.close() is called somewhere outside FireHydrant.swapSegment()"); + } + // if segment is not closed, but is same segment it means we are having trouble getting references for joinables + // of a HashJoinSegment created by segmentMapFn + return Optional.empty(); + } + segment = segmentMapFn.apply(newSinkSegment); + // Spin loop. + } + } + + @VisibleForTesting + public ReferenceCountingSegment getHydrantSegment() + { + return adapter.get(); + } + @Override public String toString() { diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java index fe6986d2f0dd..24fcb67efa29 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/AppenderatorImpl.java @@ -59,7 +59,7 @@ import org.apache.druid.segment.IndexMerger; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; @@ -756,7 +756,7 @@ private DataSegment mergeAndPush( Closer closer = Closer.create(); try { for (FireHydrant fireHydrant : sink) { - Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); + Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); log.debug("Segment[%s] adding hydrant[%s]", identifier, fireHydrant); indexes.add(queryableIndex); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java index 548de50327d0..c1226c72aa81 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BaseAppenderatorDriver.java @@ -426,9 +426,9 @@ protected AppenderatorDriverAddResult append( } /** - * Returns a stream of {@link SegmentWithState} for the given sequenceNames. + * Returns a stream of {@link SegmentIdWithShardSpec} for the given sequenceNames. */ - Stream getSegmentWithStates(Collection sequenceNames) + List getSegmentIdsWithShardSpecs(Collection sequenceNames) { synchronized (segments) { return sequenceNames @@ -436,11 +436,13 @@ Stream getSegmentWithStates(Collection sequenceNames) .map(segments::get) .filter(Objects::nonNull) .flatMap(segmentsForSequence -> segmentsForSequence.intervalToSegmentStates.values().stream()) - .flatMap(segmentsOfInterval -> segmentsOfInterval.getAllSegments().stream()); + .flatMap(segmentsOfInterval -> segmentsOfInterval.getAllSegments().stream()) + .map(SegmentWithState::getSegmentIdentifier) + .collect(Collectors.toList()); } } - Stream getAppendingSegments(Collection sequenceNames) + Set getAppendingSegments(Collection sequenceNames) { synchronized (segments) { return sequenceNames @@ -449,7 +451,9 @@ Stream getAppendingSegments(Collection sequenceNames) .filter(Objects::nonNull) .flatMap(segmentsForSequence -> segmentsForSequence.intervalToSegmentStates.values().stream()) .map(segmentsOfInterval -> segmentsOfInterval.appendingSegment) - .filter(Objects::nonNull); + .filter(Objects::nonNull) + .map(SegmentWithState::getSegmentIdentifier) + .collect(Collectors.toSet()); } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java index 4f6fd54a18e8..7be1a0b1865d 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/BatchAppenderatorDriver.java @@ -22,10 +22,11 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.util.concurrent.AsyncFunction; +import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.concurrent.ListenableFutures; import org.apache.druid.segment.loading.DataSegmentKiller; import org.apache.druid.segment.realtime.appenderator.SegmentWithState.SegmentState; import org.apache.druid.timeline.DataSegment; @@ -136,14 +137,11 @@ private SegmentsAndCommitMetadata pushAndClear( long pushAndClearTimeoutMs ) throws InterruptedException, ExecutionException, TimeoutException { - final Set requestedSegmentIdsForSequences = getAppendingSegments(sequenceNames) - .map(SegmentWithState::getSegmentIdentifier) - .collect(Collectors.toSet()); + final Set requestedSegmentIdsForSequences = getAppendingSegments(sequenceNames); - - final ListenableFuture future = ListenableFutures.transformAsync( + final ListenableFuture future = Futures.transform( pushInBackground(null, requestedSegmentIdsForSequences, false), - this::dropInBackground + (AsyncFunction) this::dropInBackground ); final SegmentsAndCommitMetadata segmentsAndCommitMetadata = diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java new file mode 100644 index 000000000000..ec4a65af3a4d --- /dev/null +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelper.java @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.realtime.appenderator; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BucketNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingShardSpec; +import org.apache.druid.timeline.partition.OverwriteShardSpec; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +public final class SegmentPublisherHelper +{ + /** + * This method fills missing information in the shard spec if necessary when publishing segments. + * + * - When time chunk lock is used, the non-appending task should set the proper size of the core partitions for + * dynamically-partitioned segments. See {@link #annotateCorePartitionSetSizeFn}. + * - When segment lock is used, the overwriting task should set the proper size of the atomic update group. + * See {@link #annotateAtomicUpdateGroupFn}. + */ + static Set annotateShardSpec(Set segments) + { + final Map> intervalToSegments = new HashMap<>(); + segments.forEach( + segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) + ); + + for (Entry> entry : intervalToSegments.entrySet()) { + final Interval interval = entry.getKey(); + final List segmentsPerInterval = entry.getValue(); + final ShardSpec firstShardSpec = segmentsPerInterval.get(0).getShardSpec(); + final boolean anyMismatch = segmentsPerInterval.stream().anyMatch( + segment -> segment.getShardSpec().getClass() != firstShardSpec.getClass() + ); + if (anyMismatch) { + throw new ISE( + "Mismatched shardSpecs in interval[%s] for segments[%s]", + interval, + segmentsPerInterval + ); + } + final Function annotateFn; + if (firstShardSpec instanceof OverwriteShardSpec) { + annotateFn = annotateAtomicUpdateGroupFn(segmentsPerInterval.size()); + } else if (firstShardSpec instanceof BuildingShardSpec) { + annotateFn = annotateCorePartitionSetSizeFn(segmentsPerInterval.size()); + } else if (firstShardSpec instanceof BucketNumberedShardSpec) { + throw new ISE("Cannot publish segments with shardSpec[%s]", firstShardSpec); + } else { + annotateFn = null; + } + + if (annotateFn != null) { + intervalToSegments.put(interval, segmentsPerInterval.stream().map(annotateFn).collect(Collectors.toList())); + } + } + + return intervalToSegments.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()); + } + + private static Function annotateAtomicUpdateGroupFn(int atomicUpdateGroupSize) + { + // The segments which are published together consist an atomicUpdateGroup. + return segment -> { + final OverwriteShardSpec shardSpec = (OverwriteShardSpec) segment.getShardSpec(); + return segment.withShardSpec(shardSpec.withAtomicUpdateGroupSize((short) atomicUpdateGroupSize)); + }; + } + + private static Function annotateCorePartitionSetSizeFn(int corePartitionSetSize) + { + return segment -> { + final BuildingShardSpec shardSpec = (BuildingShardSpec) segment.getShardSpec(); + return segment.withShardSpec(shardSpec.convert(corePartitionSetSize)); + }; + } + + private SegmentPublisherHelper() + { + } +} diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java index 504daf3d60a0..b46bd7431800 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/SinkQuerySegmentWalker.java @@ -28,7 +28,6 @@ import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.ForegroundCachePopulator; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.CloseQuietly; @@ -57,9 +56,10 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.segment.realtime.FireHydrant; import org.apache.druid.segment.realtime.plumber.Sink; import org.apache.druid.timeline.SegmentId; @@ -170,17 +170,20 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); } + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( + QueryContexts.getEnableJoinFilterPushDown(query), + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query) + ); + // segmentMapFn maps each base Segment into a joined Segment if necessary. - final Function segmentMapFn = Joinables.createSegmentMapFn( + final Function segmentMapFn = Joinables.createSegmentMapFn( analysis.getPreJoinableClauses(), joinableFactory, cpuTimeAccumulator, - QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query), - QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), - QueryContexts.getJoinFilterRewriteMaxSize(query), - query.getFilter() == null ? null : query.getFilter().toFilter(), - query.getVirtualColumns() + joinFilterRewriteConfig, + query ); Iterable> perSegmentRunners = Iterables.transform( @@ -211,15 +214,24 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final final boolean hydrantDefinitelySwapped = hydrant.hasSwapped(); if (skipIncrementalSegment && !hydrantDefinitelySwapped) { - return new Pair<>(Intervals.ETERNITY, new NoopQueryRunner<>()); + return new Pair<>(hydrant.getSegmentDataInterval(), new NoopQueryRunner<>()); } // Prevent the underlying segment from swapping when its being iterated - final Pair segmentAndCloseable = hydrant.getAndIncrementSegment(); + final Optional> maybeSegmentAndCloseable = + hydrant.getSegmentForQuery(segmentMapFn); + + // if optional isn't present, we failed to acquire reference to the segment or any joinables + if (!maybeSegmentAndCloseable.isPresent()) { + return new Pair<>( + hydrant.getSegmentDataInterval(), + new ReportTimelineMissingSegmentQueryRunner<>(descriptor) + ); + } + final Pair segmentAndCloseable = maybeSegmentAndCloseable.get(); try { - final Segment mappedSegment = segmentMapFn.apply(segmentAndCloseable.lhs); - QueryRunner runner = factory.createRunner(mappedSegment); + QueryRunner runner = factory.createRunner(segmentAndCloseable.lhs); // 1) Only use caching if data is immutable // 2) Hydrants are not the same between replicas, make sure cache is local @@ -245,7 +257,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final runner, segmentAndCloseable.rhs ); - return new Pair<>(mappedSegment.getDataInterval(), runner); + return new Pair<>(segmentAndCloseable.lhs.getDataInterval(), runner); } catch (RuntimeException e) { CloseQuietly.close(segmentAndCloseable.rhs); diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java index bee2c7dc9932..0555f2fd37ea 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/StreamAppenderatorDriver.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.util.concurrent.AsyncFunction; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -32,7 +33,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ListenableFutures; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.SegmentDescriptor; @@ -271,15 +271,13 @@ public ListenableFuture publish( final Collection sequenceNames ) { - final List theSegments = getSegmentWithStates(sequenceNames) - .map(SegmentWithState::getSegmentIdentifier) - .collect(Collectors.toList()); + final List theSegments = getSegmentIdsWithShardSpecs(sequenceNames); - final ListenableFuture publishFuture = ListenableFutures.transformAsync( + final ListenableFuture publishFuture = Futures.transform( // useUniquePath=true prevents inconsistencies in segment data when task failures or replicas leads to a second // version of a segment with the same identifier containing different data; see DataSegmentPusher.push() docs pushInBackground(wrapCommitter(committer), theSegments, true), - sam -> publishInBackground( + (AsyncFunction) sam -> publishInBackground( null, sam, publisher @@ -388,9 +386,9 @@ public ListenableFuture publishAndRegisterHandoff( final Collection sequenceNames ) { - return ListenableFutures.transformAsync( + return Futures.transform( publish(publisher, committer, sequenceNames), - this::registerHandoff + (AsyncFunction) this::registerHandoff ); } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java index cb9b9ffb4448..26d8fac9dff5 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/appenderator/TransactionalSegmentPublisher.java @@ -20,21 +20,11 @@ package org.apache.druid.segment.realtime.appenderator; import org.apache.druid.indexing.overlord.SegmentPublishResult; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.OverwriteShardSpec; -import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; import java.util.Set; -import java.util.stream.Collectors; public interface TransactionalSegmentPublisher { @@ -63,7 +53,7 @@ default SegmentPublishResult publishSegments( { return publishAnnotatedSegments( segmentsToBeOverwritten, - annotateAtomicUpdateGroupSize(segmentsToPublish), + SegmentPublisherHelper.annotateShardSpec(segmentsToPublish), commitMetadata ); } @@ -76,52 +66,4 @@ default boolean supportsEmptyPublish() { return false; } - - static Set annotateAtomicUpdateGroupSize(Set segments) - { - final Map> intervalToSegments = new HashMap<>(); - segments.forEach( - segment -> intervalToSegments.computeIfAbsent(segment.getInterval(), k -> new ArrayList<>()).add(segment) - ); - - for (Entry> entry : intervalToSegments.entrySet()) { - final Interval interval = entry.getKey(); - final List segmentsPerInterval = entry.getValue(); - final boolean isNonFirstGeneration = segmentsPerInterval.get(0).getShardSpec() instanceof OverwriteShardSpec; - - final boolean anyMismatch = segmentsPerInterval.stream().anyMatch( - segment -> (segment.getShardSpec() instanceof OverwriteShardSpec) != isNonFirstGeneration - ); - if (anyMismatch) { - throw new ISE( - "WTH? some segments have empty overshadwedSegments but others are not? " - + "segments with non-overwritingShardSpec: [%s]," - + "segments with overwritingShardSpec: [%s]", - segmentsPerInterval.stream() - .filter(segment -> !(segment.getShardSpec() instanceof OverwriteShardSpec)) - .collect(Collectors.toList()), - segmentsPerInterval.stream() - .filter(segment -> segment.getShardSpec() instanceof OverwriteShardSpec) - .collect(Collectors.toList()) - ); - } - - if (isNonFirstGeneration) { - // The segments which are published together consist an atomicUpdateGroup. - - intervalToSegments.put( - interval, - segmentsPerInterval - .stream() - .map(segment -> { - final OverwriteShardSpec shardSpec = (OverwriteShardSpec) segment.getShardSpec(); - return segment.withShardSpec(shardSpec.withAtomicUpdateGroupSize((short) segmentsPerInterval.size())); - }) - .collect(Collectors.toList()) - ); - } - } - - return intervalToSegments.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()); - } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java index 9daa231f9b7e..0b5863d671d6 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactory.java @@ -22,32 +22,23 @@ import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import org.apache.druid.data.input.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; -import org.skife.jdbi.v2.ResultIterator; -import org.skife.jdbi.v2.exceptions.CallbackFailedException; -import org.skife.jdbi.v2.exceptions.ResultSetException; -import org.skife.jdbi.v2.exceptions.StatementException; +import org.apache.druid.metadata.input.SqlEntity; import javax.annotation.Nullable; import java.io.File; import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.InputStream; -import java.sql.ResultSetMetaData; -import java.sql.SQLException; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -86,7 +77,10 @@ public SqlFirehoseFactory( this.sqls = sqls; this.objectMapper = objectMapper; - this.sqlFirehoseDatabaseConnector = sqlFirehoseDatabaseConnector; + this.sqlFirehoseDatabaseConnector = Preconditions.checkNotNull( + sqlFirehoseDatabaseConnector, + "SQL Metadata Connector not configured!" + ); this.foldCase = foldCase; this.connectorConfig = null; } @@ -94,79 +88,8 @@ public SqlFirehoseFactory( @Override protected InputStream openObjectStream(String sql, File fileName) throws IOException { - Preconditions.checkNotNull(sqlFirehoseDatabaseConnector, "SQL Metadata Connector not configured!"); - try (FileOutputStream fos = new FileOutputStream(fileName)) { - final JsonGenerator jg = objectMapper.getFactory().createGenerator(fos); - sqlFirehoseDatabaseConnector.retryWithHandle( - (handle) -> { - ResultIterator> resultIterator = handle.createQuery( - sql - ).map( - (index, r, ctx) -> { - Map resultRow = foldCase ? new CaseFoldedMap() : new HashMap<>(); - ResultSetMetaData resultMetadata; - try { - resultMetadata = r.getMetaData(); - } - catch (SQLException e) { - throw new ResultSetException("Unable to obtain metadata from result set", e, ctx); - } - try { - for (int i = 1; i <= resultMetadata.getColumnCount(); i++) { - String key = resultMetadata.getColumnName(i); - String alias = resultMetadata.getColumnLabel(i); - Object value = r.getObject(i); - resultRow.put(alias != null ? alias : key, value); - } - } - catch (SQLException e) { - throw new ResultSetException("Unable to access specific metadata from " + - "result set metadata", e, ctx); - } - return resultRow; - } - ).iterator(); - jg.writeStartArray(); - while (resultIterator.hasNext()) { - jg.writeObject(resultIterator.next()); - } - jg.writeEndArray(); - jg.close(); - return null; - }, - (exception) -> { - final boolean isStatementException = exception instanceof StatementException || - (exception instanceof CallbackFailedException - && exception.getCause() instanceof StatementException); - return sqlFirehoseDatabaseConnector.isTransientException(exception) && !(isStatementException); - } - ); - } + SqlEntity.openCleanableFile(sql, sqlFirehoseDatabaseConnector, objectMapper, foldCase, fileName); return new FileInputStream(fileName); - - } - - private static class CaseFoldedMap extends HashMap - { - public static final long serialVersionUID = 1L; - - @Override - public Object get(Object obj) - { - return super.get(StringUtils.toLowerCase((String) obj)); - } - - @Override - public Object put(String key, Object value) - { - return super.put(StringUtils.toLowerCase(key), value); - } - - @Override - public boolean containsKey(Object obj) - { - return super.containsKey(StringUtils.toLowerCase((String) obj)); - } } @Override diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java index bbee720a88ce..2e97258a52da 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifier.java @@ -142,7 +142,7 @@ static boolean isHandOffComplete(List serverView, Segm && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 - && segmentLoadInfo.getServers().stream().anyMatch(DruidServerMetadata::segmentReplicatable)) { + && segmentLoadInfo.getServers().stream().anyMatch(DruidServerMetadata::isSegmentReplicationOrBroadcastTarget)) { return true; } } diff --git a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java index 7a7db66d1f90..1fdc0b0f6804 100644 --- a/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java +++ b/server/src/main/java/org/apache/druid/segment/realtime/plumber/RealtimePlumber.java @@ -56,7 +56,7 @@ import org.apache.druid.segment.Metadata; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.incremental.IncrementalIndexAddResult; import org.apache.druid.segment.incremental.IndexSizeExceededException; import org.apache.druid.segment.indexing.DataSchema; @@ -424,7 +424,7 @@ public void doRun() Closer closer = Closer.create(); try { for (FireHydrant fireHydrant : sink) { - Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); + Pair segmentAndCloseable = fireHydrant.getAndIncrementSegment(); final QueryableIndex queryableIndex = segmentAndCloseable.lhs.asQueryableIndex(); log.info("Adding hydrant[%s]", fireHydrant); indexes.add(queryableIndex); 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 fa35ff7b8a9c..c7318a60f7f6 100644 --- a/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/ClientQuerySegmentWalker.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.DataSource; import org.apache.druid.query.FluentQueryRunnerBuilder; +import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.PostProcessingOperator; import org.apache.druid.query.Query; @@ -47,9 +48,11 @@ import org.apache.druid.query.RetryQueryRunner; import org.apache.druid.query.RetryQueryRunnerConfig; import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.server.initialization.ServerConfig; import org.joda.time.Interval; @@ -77,6 +80,7 @@ public class ClientQuerySegmentWalker implements QuerySegmentWalker private final QuerySegmentWalker clusterClient; private final QuerySegmentWalker localClient; private final QueryToolChestWarehouse warehouse; + private final JoinableFactory joinableFactory; private final RetryQueryRunnerConfig retryConfig; private final ObjectMapper objectMapper; private final ServerConfig serverConfig; @@ -88,6 +92,7 @@ public ClientQuerySegmentWalker( QuerySegmentWalker clusterClient, QuerySegmentWalker localClient, QueryToolChestWarehouse warehouse, + JoinableFactory joinableFactory, RetryQueryRunnerConfig retryConfig, ObjectMapper objectMapper, ServerConfig serverConfig, @@ -99,6 +104,7 @@ public ClientQuerySegmentWalker( this.clusterClient = clusterClient; this.localClient = localClient; this.warehouse = warehouse; + this.joinableFactory = joinableFactory; this.retryConfig = retryConfig; this.objectMapper = objectMapper; this.serverConfig = serverConfig; @@ -112,6 +118,7 @@ public ClientQuerySegmentWalker( CachingClusteredClient clusterClient, LocalQuerySegmentWalker localClient, QueryToolChestWarehouse warehouse, + JoinableFactory joinableFactory, RetryQueryRunnerConfig retryConfig, ObjectMapper objectMapper, ServerConfig serverConfig, @@ -124,6 +131,7 @@ public ClientQuerySegmentWalker( (QuerySegmentWalker) clusterClient, (QuerySegmentWalker) localClient, warehouse, + joinableFactory, retryConfig, objectMapper, serverConfig, @@ -137,10 +145,13 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable> toolChest = warehouse.getToolChest(query); - // First, do an inlining dry run to see if any inlining is necessary, without actually running the queries. + // transform TableDataSource to GlobalTableDataSource when eligible + // before further transformation to potentially inline + final DataSource freeTradeDataSource = globalizeIfPossible(query.getDataSource()); + // do an inlining dry run to see if any inlining is necessary, without actually running the queries. final int maxSubqueryRows = QueryContexts.getMaxSubqueryRows(query, serverConfig.getMaxSubqueryRows()); final DataSource inlineDryRun = inlineIfNecessary( - query.getDataSource(), + freeTradeDataSource, toolChest, new AtomicInteger(), maxSubqueryRows, @@ -156,7 +167,7 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable newQuery = query.withDataSource( inlineIfNecessary( - query.getDataSource(), + freeTradeDataSource, toolChest, new AtomicInteger(), maxSubqueryRows, @@ -187,10 +198,15 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) { - // Inlining isn't done for segments-based queries. + // Inlining isn't done for segments-based queries, but we still globalify the table datasources if possible + final Query freeTradeQuery = query.withDataSource(globalizeIfPossible(query.getDataSource())); if (canRunQueryUsingClusterWalker(query)) { - return decorateClusterRunner(query, clusterClient.getQueryRunnerForSegments(query, specs)); + return new QuerySwappingQueryRunner<>( + decorateClusterRunner(freeTradeQuery, clusterClient.getQueryRunnerForSegments(freeTradeQuery, specs)), + query, + freeTradeQuery + ); } else { // We don't expect end-users to see this message, since it only happens when specific segments are requested; // this is not typical end-user behavior. @@ -235,6 +251,27 @@ private boolean canRunQueryUsingClusterWalker(Query query) || toolChest.canPerformSubquery(((QueryDataSource) analysis.getDataSource()).getQuery())); } + + private DataSource globalizeIfPossible( + final DataSource dataSource + ) + { + if (dataSource instanceof TableDataSource) { + GlobalTableDataSource maybeGlobal = new GlobalTableDataSource(((TableDataSource) dataSource).getName()); + if (joinableFactory.isDirectlyJoinable(maybeGlobal)) { + return maybeGlobal; + } + return dataSource; + } else { + List currentChildren = dataSource.getChildren(); + List newChildren = new ArrayList<>(currentChildren.size()); + for (DataSource child : currentChildren) { + newChildren.add(globalizeIfPossible(child)); + } + return dataSource.withChildren(newChildren); + } + } + /** * Replace QueryDataSources with InlineDataSources when necessary and possible. "Necessary" is defined as: * diff --git a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java index d7f39adaa4ba..dbd043f52bf2 100644 --- a/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java +++ b/server/src/main/java/org/apache/druid/server/LocalQuerySegmentWalker.java @@ -24,6 +24,7 @@ import org.apache.druid.java.util.common.IAE; 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.FunctionalIterable; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.FluentQueryRunnerBuilder; import org.apache.druid.query.Query; @@ -35,10 +36,13 @@ import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.SegmentWrangler; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.joda.time.Interval; import java.util.HashSet; @@ -89,20 +93,26 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final throw new IAE("Cannot query dataSource locally: %s", analysis.getDataSource()); } - final Iterable segments = segmentWrangler.getSegmentsForIntervals(analysis.getBaseDataSource(), intervals); + // wrap in ReferenceCountingSegment, these aren't currently managed by SegmentManager so reference tracking doesn't + // matter, but at least some or all will be in a future PR + final Iterable segments = + FunctionalIterable.create(segmentWrangler.getSegmentsForIntervals(analysis.getBaseDataSource(), intervals)) + .transform(ReferenceCountingSegment::wrapRootGenerationSegment); final Query prioritizedAndLaned = prioritizeAndLaneQuery(query, segments); final AtomicLong cpuAccumulator = new AtomicLong(0L); - final Function segmentMapFn = Joinables.createSegmentMapFn( - analysis.getPreJoinableClauses(), - joinableFactory, - cpuAccumulator, + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( QueryContexts.getEnableJoinFilterPushDown(prioritizedAndLaned), QueryContexts.getEnableJoinFilterRewrite(prioritizedAndLaned), QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(prioritizedAndLaned), - QueryContexts.getJoinFilterRewriteMaxSize(prioritizedAndLaned), - prioritizedAndLaned.getFilter() == null ? null : prioritizedAndLaned.getFilter().toFilter(), - prioritizedAndLaned.getVirtualColumns() + QueryContexts.getJoinFilterRewriteMaxSize(prioritizedAndLaned) + ); + final Function segmentMapFn = Joinables.createSegmentMapFn( + analysis.getPreJoinableClauses(), + joinableFactory, + cpuAccumulator, + joinFilterRewriteConfig, + query ); final QueryRunnerFactory> queryRunnerFactory = conglomerate.findFactory(prioritizedAndLaned); @@ -122,6 +132,7 @@ public QueryRunner getQueryRunnerForIntervals(final Query query, final .applyPostMergeDecoration() .emitCPUTimeMetric(emitter, cpuAccumulator); } + @Override public QueryRunner getQueryRunnerForSegments(final Query query, final Iterable specs) { @@ -129,7 +140,7 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final throw new ISE("Cannot run with specific segments"); } - private Query prioritizeAndLaneQuery(Query query, Iterable segments) + private Query prioritizeAndLaneQuery(Query query, Iterable segments) { Set segmentServerSelectors = new HashSet<>(); for (Segment s : segments) { 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 d270799b62af..131ebf134f45 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -44,6 +44,7 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryToolChest; +import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.server.metrics.QueryCountStatsProvider; import org.apache.druid.server.security.Access; @@ -315,6 +316,11 @@ public void write(OutputStream outputStream) throws WebApplicationException queryLifecycle.emitLogsAndMetrics(cap, req.getRemoteAddr(), -1); return ioReaderWriter.gotLimited(cap); } + catch (QueryUnsupportedException unsupported) { + failedQueryCount.incrementAndGet(); + queryLifecycle.emitLogsAndMetrics(unsupported, req.getRemoteAddr(), -1); + return ioReaderWriter.gotUnsupported(unsupported); + } catch (ForbiddenException e) { // don't do anything for an authorization failure, ForbiddenExceptionMapper will catch this later and // send an error response if this is thrown. @@ -446,6 +452,13 @@ Response gotLimited(QueryCapacityExceededException e) throws IOException .entity(newOutputWriter(null, null, false).writeValueAsBytes(e)) .build(); } + + Response gotUnsupported(QueryUnsupportedException e) throws IOException + { + return Response.status(QueryUnsupportedException.STATUS_CODE) + .entity(newOutputWriter(null, null, false).writeValueAsBytes(e)) + .build(); + } } @Override 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 45b6538bb610..a5484f21c9a7 100644 --- a/server/src/main/java/org/apache/druid/server/SegmentManager.java +++ b/server/src/main/java/org/apache/druid/server/SegmentManager.java @@ -40,6 +40,7 @@ import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; /** @@ -121,6 +122,11 @@ public Map getDataSourceSizes() return CollectionUtils.mapValues(dataSources, SegmentManager.DataSourceState::getTotalSegmentSize); } + public Set getDataSourceNames() + { + return dataSources.keySet(); + } + /** * Returns a map of dataSource to the number of segments managed by this segmentManager. This method should be * carefully because the returned map might be different from the actual data source states. diff --git a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java index e3673bbc9cae..3fda41b08dab 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DruidServerMetadata.java @@ -107,11 +107,21 @@ public int getPriority() return priority; } - public boolean segmentReplicatable() + public boolean isSegmentReplicationTarget() { return type.isSegmentReplicationTarget(); } + public boolean isSegmentBroadcastTarget() + { + return type.isSegmentBroadcastTarget(); + } + + public boolean isSegmentReplicationOrBroadcastTarget() + { + return isSegmentReplicationTarget() || isSegmentBroadcastTarget(); + } + @Override public boolean equals(Object o) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.java new file mode 100644 index 000000000000..4f4f7a5b1d19 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/coordination/LoadableDataSegment.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.server.coordination; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import org.apache.druid.jackson.CommaListJoinDeserializer; +import org.apache.druid.timeline.CompactionState; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.ShardSpec; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.Map; + +/** + * A deserialization aid used by {@link SegmentChangeRequestLoad}. The broker prunes the loadSpec from segments + * for efficiency reasons, but the broker does need the loadSpec when it loads broadcast segments. + * + * This class always uses the non-pruning default {@link PruneSpecsHolder}. + */ +public class LoadableDataSegment extends DataSegment +{ + @JsonCreator + public LoadableDataSegment( + @JsonProperty("dataSource") String dataSource, + @JsonProperty("interval") Interval interval, + @JsonProperty("version") String version, + // use `Map` *NOT* `LoadSpec` because we want to do lazy materialization to prevent dependency pollution + @JsonProperty("loadSpec") @Nullable Map loadSpec, + @JsonProperty("dimensions") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List dimensions, + @JsonProperty("metrics") + @JsonDeserialize(using = CommaListJoinDeserializer.class) + @Nullable + List metrics, + @JsonProperty("shardSpec") @Nullable ShardSpec shardSpec, + @JsonProperty("lastCompactionState") @Nullable CompactionState lastCompactionState, + @JsonProperty("binaryVersion") Integer binaryVersion, + @JsonProperty("size") long size, + @JacksonInject PruneSpecsHolder pruneSpecsHolder + ) + { + super( + dataSource, + interval, + version, + loadSpec, + dimensions, + metrics, + shardSpec, + lastCompactionState, + binaryVersion, + size, + PruneSpecsHolder.DEFAULT + ); + + } +} diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java index 097e02523032..130c7b50d80c 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentChangeRequestLoad.java @@ -35,14 +35,26 @@ public class SegmentChangeRequestLoad implements DataSegmentChangeRequest { private final DataSegment segment; + /** + * To avoid pruning of the loadSpec on the broker, needed when the broker is loading broadcast segments, + * we deserialize into an {@link LoadableDataSegment}, which never removes the loadSpec. + */ @JsonCreator public SegmentChangeRequestLoad( - @JsonUnwrapped DataSegment segment + @JsonUnwrapped LoadableDataSegment segment ) { this.segment = segment; } + public SegmentChangeRequestLoad( + DataSegment segment + ) + { + this.segment = segment; + } + + @Override public void go(DataSegmentChangeHandler handler, @Nullable DataSegmentChangeCallback callback) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 51aefc1b8d67..87a19365e6f5 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -34,6 +34,8 @@ import com.google.common.util.concurrent.SettableFuture; import com.google.inject.Inject; import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; @@ -104,7 +106,8 @@ public SegmentLoadDropHandler( SegmentLoaderConfig config, DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, - SegmentManager segmentManager + SegmentManager segmentManager, + ServerTypeConfig serverTypeConfig ) { this( @@ -116,7 +119,8 @@ public SegmentLoadDropHandler( Executors.newScheduledThreadPool( config.getNumLoadingThreads(), Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") - ) + ), + serverTypeConfig ); } @@ -127,7 +131,8 @@ public SegmentLoadDropHandler( DataSegmentAnnouncer announcer, DataSegmentServerAnnouncer serverAnnouncer, SegmentManager segmentManager, - ScheduledExecutorService exec + ScheduledExecutorService exec, + ServerTypeConfig serverTypeConfig ) { this.jsonMapper = jsonMapper; @@ -139,6 +144,13 @@ public SegmentLoadDropHandler( this.exec = exec; this.segmentsToDelete = new ConcurrentSkipListSet<>(); + if (config.getLocations().isEmpty()) { + if (ServerType.HISTORICAL.equals(serverTypeConfig.getServerType())) { + throw new IAE("Segment cache locations must be set on historicals."); + } else { + log.info("Not starting SegmentLoadDropHandler with empty segment cache locations."); + } + } requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); } @@ -152,8 +164,10 @@ public void start() throws IOException log.info("Starting..."); try { - loadLocalCache(); - serverAnnouncer.announce(); + if (!config.getLocations().isEmpty()) { + loadLocalCache(); + serverAnnouncer.announce(); + } } catch (Exception e) { Throwables.propagateIfPossible(e, IOException.class); @@ -174,7 +188,9 @@ public void stop() log.info("Stopping..."); try { - serverAnnouncer.unannounce(); + if (!config.getLocations().isEmpty()) { + serverAnnouncer.unannounce(); + } } catch (Exception e) { throw new RuntimeException(e); 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 ca26e0dd4f75..b2bef3302623 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 @@ -53,11 +53,11 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.SpecificSegmentQueryRunner; import org.apache.druid.query.spec.SpecificSegmentSpec; -import org.apache.druid.segment.ReferenceCounter; import org.apache.druid.segment.ReferenceCountingSegment; -import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.SetAndVerifyContextQueryRunner; import org.apache.druid.server.initialization.ServerConfig; @@ -193,17 +193,20 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable(); } + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( + QueryContexts.getEnableJoinFilterPushDown(query), + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query) + ); + // segmentMapFn maps each base Segment into a joined Segment if necessary. - final Function segmentMapFn = Joinables.createSegmentMapFn( + final Function segmentMapFn = Joinables.createSegmentMapFn( analysis.getPreJoinableClauses(), joinableFactory, cpuTimeAccumulator, - QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query), - QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), - QueryContexts.getJoinFilterRewriteMaxSize(query), - query.getFilter() == null ? null : query.getFilter().toFilter(), - query.getVirtualColumns() + joinFilterRewriteConfig, + query ); FunctionalIterable> queryRunners = FunctionalIterable @@ -230,7 +233,6 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable QueryRunner getQueryRunnerForSegments(Query query, Iterable QueryRunner buildAndDecorateQueryRunner( final QueryRunnerFactory> factory, final QueryToolChest> toolChest, - final Segment segment, - final ReferenceCounter segmentReferenceCounter, + final SegmentReference segment, final SegmentDescriptor segmentDescriptor, final AtomicLong cpuTimeAccumulator ) @@ -266,7 +267,7 @@ private QueryRunner buildAndDecorateQueryRunner( MetricsEmittingQueryRunner metricsEmittingQueryRunnerInner = new MetricsEmittingQueryRunner<>( emitter, toolChest, - new ReferenceCountingSegmentQueryRunner<>(factory, segment, segmentReferenceCounter, segmentDescriptor), + new ReferenceCountingSegmentQueryRunner<>(factory, segment, segmentDescriptor), QueryMetrics::reportSegmentTime, queryMetrics -> queryMetrics.segment(segmentIdString) ); diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerType.java b/server/src/main/java/org/apache/druid/server/coordination/ServerType.java index 42fb65a3fdfb..0b860a1b0afe 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerType.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerType.java @@ -63,6 +63,14 @@ public boolean isSegmentReplicationTarget() { return false; } + }, + + BROKER { + @Override + public boolean isSegmentReplicationTarget() + { + return false; + } }; /** diff --git a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java index d9fea81e3a95..889c167c8ef5 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/BalancerStrategy.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.NavigableSet; +import java.util.Set; /** * This interface describes the coordinator balancing strategy, which is responsible for making decisions on where @@ -56,11 +57,17 @@ public interface BalancerStrategy /** * Pick the best segment to move from one of the supplied set of servers according to the balancing strategy. * @param serverHolders set of historicals to consider for moving segments + * @param broadcastDatasources Datasources that contain segments which were loaded via broadcast rules. + * Balancing strategies should avoid rebalancing segments for such datasources, since + * they should be loaded on all servers anyway. + * NOTE: this should really be handled on a per-segment basis, to properly support + * the interval or period-based broadcast rules. For simplicity of the initial + * implementation, only forever broadcast rules are supported. * @return {@link BalancerSegmentHolder} containing segment to move and server it currently resides on, or null if * there are no segments to pick from (i. e. all provided serverHolders are empty). */ @Nullable - BalancerSegmentHolder pickSegmentToMove(List serverHolders); + BalancerSegmentHolder pickSegmentToMove(List serverHolders, Set broadcastDatasources); /** * Returns an iterator for a set of servers to drop from, ordered by preference of which server to drop from first diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java index 4a1989df24df..1741087e8c5c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CachingCostBalancerStrategyFactory.java @@ -71,7 +71,7 @@ public CachingCostBalancerStrategyFactory( @Override public ServerView.CallbackAction segmentAdded(DruidServerMetadata server, DataSegment segment) { - if (server.segmentReplicatable()) { + if (server.isSegmentReplicationTarget()) { clusterCostCacheBuilder.addSegment(server.getName(), segment); } return ServerView.CallbackAction.CONTINUE; @@ -80,7 +80,7 @@ public ServerView.CallbackAction segmentAdded(DruidServerMetadata server, DataSe @Override public ServerView.CallbackAction segmentRemoved(DruidServerMetadata server, DataSegment segment) { - if (server.segmentReplicatable()) { + if (server.isSegmentReplicationTarget()) { clusterCostCacheBuilder.removeSegment(server.getName(), segment); } return ServerView.CallbackAction.CONTINUE; @@ -98,7 +98,7 @@ public ServerView.CallbackAction segmentViewInitialized() serverInventoryView.registerServerRemovedCallback( executor, server -> { - if (server.segmentReplicatable()) { + if (server.isSegmentReplicationTarget()) { clusterCostCacheBuilder.removeServer(server.getName()); } return ServerView.CallbackAction.CONTINUE; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java index 4fd4164f3001..5d656d643f99 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java @@ -35,6 +35,7 @@ import java.util.Iterator; import java.util.List; import java.util.NavigableSet; +import java.util.Set; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; @@ -211,9 +212,12 @@ static double computeJointSegmentsCost(final DataSegment segment, final Iterable @Override - public BalancerSegmentHolder pickSegmentToMove(final List serverHolders) + public BalancerSegmentHolder pickSegmentToMove( + final List serverHolders, + Set broadcastDatasources + ) { - return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders); + return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders, broadcastDatasources); } @Override diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java index 318f663609b0..8fb4ccb056d4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java @@ -47,24 +47,28 @@ public class DruidCluster @VisibleForTesting static DruidCluster createDruidClusterFromBuilderInTest( @Nullable Set realtimes, - Map> historicals + Map> historicals, + @Nullable Set brokers ) { - return new DruidCluster(realtimes, historicals); + return new DruidCluster(realtimes, historicals, brokers); } private final Set realtimes; private final Map> historicals; + private final Set brokers; public DruidCluster() { this.realtimes = new HashSet<>(); this.historicals = new HashMap<>(); + this.brokers = new HashSet<>(); } private DruidCluster( @Nullable Set realtimes, - Map> historicals + Map> historicals, + @Nullable Set brokers ) { this.realtimes = realtimes == null ? new HashSet<>() : new HashSet<>(realtimes); @@ -72,6 +76,7 @@ private DruidCluster( historicals, holders -> CollectionUtils.newTreeSet(Comparator.reverseOrder(), holders) ); + this.brokers = brokers == null ? new HashSet<>() : new HashSet<>(brokers); } public void add(ServerHolder serverHolder) @@ -87,7 +92,11 @@ public void add(ServerHolder serverHolder) addHistorical(serverHolder); break; case INDEXER_EXECUTOR: - throw new IAE("unsupported server type[%s]", serverHolder.getServer().getType()); + addRealtime(serverHolder); + break; + case BROKER: + addBroker(serverHolder); + break; default: throw new IAE("unknown server type[%s]", serverHolder.getServer().getType()); } @@ -108,6 +117,11 @@ private void addHistorical(ServerHolder serverHolder) tierServers.add(serverHolder); } + private void addBroker(ServerHolder serverHolder) + { + brokers.add(serverHolder); + } + public Set getRealtimes() { return realtimes; @@ -118,6 +132,12 @@ public Map> getHistoricals() return historicals; } + + public Set getBrokers() + { + return brokers; + } + public Iterable getTierNames() { return historicals.keySet(); @@ -135,6 +155,7 @@ public Collection getAllServers() final List allServers = new ArrayList<>(historicalSize + realtimeSize); historicals.values().forEach(allServers::addAll); + allServers.addAll(brokers); allServers.addAll(realtimes); return allServers; } @@ -146,7 +167,7 @@ public Iterable> getSortedHistoricalsByTier() public boolean isEmpty() { - return historicals.isEmpty() && realtimes.isEmpty(); + return historicals.isEmpty() && realtimes.isEmpty() && brokers.isEmpty(); } public boolean hasHistoricals() @@ -159,9 +180,19 @@ public boolean hasRealtimes() return !realtimes.isEmpty(); } + public boolean hasBrokers() + { + return !brokers.isEmpty(); + } + public boolean hasTier(String tier) { - NavigableSet servers = historicals.get(tier); - return (servers != null) && !servers.isEmpty(); + NavigableSet historicalServers = historicals.get(tier); + boolean historicalsHasTier = (historicalServers != null) && !historicalServers.isEmpty(); + if (historicalsHasTier) { + return true; + } + + return false; } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index f8c3f43c76f5..64168e11d76b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -29,7 +29,6 @@ import it.unimi.dsi.fastutil.objects.Object2IntMaps; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; -import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.utils.ZKPaths; import org.apache.druid.client.DataSourcesSnapshot; @@ -256,6 +255,22 @@ public Map getLoadManagementPeons() * @return tier -> { dataSource -> underReplicationCount } map */ public Map> computeUnderReplicationCountsPerDataSourcePerTier() + { + final Iterable dataSegments = segmentsMetadataManager.iterateAllUsedSegments(); + return computeUnderReplicationCountsPerDataSourcePerTierForSegments(dataSegments); + } + + /** + * segmentReplicantLookup use in this method could potentially be stale since it is only updated on coordinator runs. + * However, this is ok as long as the {@param dataSegments} is refreshed/latest as this would at least still ensure + * that the stale data in segmentReplicantLookup would be under counting replication levels, + * rather than potentially falsely reporting that everything is available. + * + * @return tier -> { dataSource -> underReplicationCount } map + */ + public Map> computeUnderReplicationCountsPerDataSourcePerTierForSegments( + Iterable dataSegments + ) { final Map> underReplicationCountsPerDataSourcePerTier = new HashMap<>(); @@ -263,28 +278,27 @@ public Map> computeUnderReplicationCountsPerDataS return underReplicationCountsPerDataSourcePerTier; } - final Iterable dataSegments = segmentsMetadataManager.iterateAllUsedSegments(); - final DateTime now = DateTimes.nowUtc(); for (final DataSegment segment : dataSegments) { final List rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource()); for (final Rule rule : rules) { - if (!(rule instanceof LoadRule && rule.appliesTo(segment, now))) { + if (!rule.appliesTo(segment, now)) { + // Rule did not match. Continue to the next Rule. continue; } + if (!rule.canLoadSegments()) { + // Rule matched but rule does not and cannot load segments. + // Hence, there is no need to update underReplicationCountsPerDataSourcePerTier map + break; + } + + rule.updateUnderReplicated(underReplicationCountsPerDataSourcePerTier, segmentReplicantLookup, segment); - ((LoadRule) rule) - .getTieredReplicants() - .forEach((final String tier, final Integer ruleReplicants) -> { - int currentReplicants = segmentReplicantLookup.getLoadedReplicants(segment.getId(), tier); - Object2LongMap underReplicationPerDataSource = underReplicationCountsPerDataSourcePerTier - .computeIfAbsent(tier, ignored -> new Object2LongOpenHashMap<>()); - ((Object2LongOpenHashMap) underReplicationPerDataSource) - .addTo(segment.getDataSource(), Math.max(ruleReplicants - currentReplicants, 0)); - }); - break; // only the first matching rule applies + // Only the first matching rule applies. This is because the Coordinator cycle through all used segments + // and match each segment with the first rule that applies. Each segment may only match a single rule. + break; } } @@ -320,7 +334,7 @@ public Map getLoadStatus() for (ImmutableDruidDataSource dataSource : dataSources) { final Set segments = Sets.newHashSet(dataSource.getSegments()); - final int numUsedSegments = segments.size(); + final int numPublishedSegments = segments.size(); // remove loaded segments for (DruidServer druidServer : serverInventoryView.getInventory()) { @@ -333,10 +347,10 @@ public Map getLoadStatus() } } } - final int numUnloadedSegments = segments.size(); + final int numUnavailableSegments = segments.size(); loadStatus.put( dataSource.getName(), - 100 * ((double) (numUsedSegments - numUnloadedSegments) / (double) numUsedSegments) + 100 * ((double) (numPublishedSegments - numUnavailableSegments) / (double) numPublishedSegments) ); } @@ -761,7 +775,7 @@ List prepareCurrentServers() List currentServers = serverInventoryView .getInventory() .stream() - .filter(DruidServer::segmentReplicatable) + .filter(DruidServer::isSegmentReplicationOrBroadcastTarget) .map(DruidServer::toImmutableDruidServer) .collect(Collectors.toList()); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index 7bae11e61db0..d73febeca0e3 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -34,7 +34,9 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; +import java.util.Set; import java.util.TreeSet; import java.util.concurrent.TimeUnit; @@ -70,6 +72,7 @@ private static TreeSet createUsedSegmentsSet(Iterable private final CoordinatorStats stats; private final DateTime balancerReferenceTimestamp; private final BalancerStrategy balancerStrategy; + private final Set broadcastDatasources; private DruidCoordinatorRuntimeParams( long startTimeNanos, @@ -85,7 +88,8 @@ private DruidCoordinatorRuntimeParams( CoordinatorCompactionConfig coordinatorCompactionConfig, CoordinatorStats stats, DateTime balancerReferenceTimestamp, - BalancerStrategy balancerStrategy + BalancerStrategy balancerStrategy, + Set broadcastDatasources ) { this.startTimeNanos = startTimeNanos; @@ -102,6 +106,7 @@ private DruidCoordinatorRuntimeParams( this.stats = stats; this.balancerReferenceTimestamp = balancerReferenceTimestamp; this.balancerStrategy = balancerStrategy; + this.broadcastDatasources = broadcastDatasources; } public long getStartTimeNanos() @@ -180,6 +185,11 @@ public BalancerStrategy getBalancerStrategy() return balancerStrategy; } + public Set getBroadcastDatasources() + { + return broadcastDatasources; + } + public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements() { long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos(); @@ -216,7 +226,8 @@ public Builder buildFromExisting() coordinatorCompactionConfig, stats, balancerReferenceTimestamp, - balancerStrategy + balancerStrategy, + broadcastDatasources ); } @@ -236,7 +247,8 @@ public Builder buildFromExistingWithoutSegmentsMetadata() coordinatorCompactionConfig, stats, balancerReferenceTimestamp, - balancerStrategy + balancerStrategy, + broadcastDatasources ); } @@ -256,6 +268,7 @@ public static class Builder private CoordinatorStats stats; private DateTime balancerReferenceTimestamp; private BalancerStrategy balancerStrategy; + private Set broadcastDatasources; private Builder() { @@ -272,6 +285,7 @@ private Builder() this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty(); this.balancerReferenceTimestamp = DateTimes.nowUtc(); + this.broadcastDatasources = new HashSet<>(); } Builder( @@ -288,7 +302,8 @@ private Builder() CoordinatorCompactionConfig coordinatorCompactionConfig, CoordinatorStats stats, DateTime balancerReferenceTimestamp, - BalancerStrategy balancerStrategy + BalancerStrategy balancerStrategy, + Set broadcastDatasources ) { this.startTimeNanos = startTimeNanos; @@ -305,6 +320,7 @@ private Builder() this.stats = stats; this.balancerReferenceTimestamp = balancerReferenceTimestamp; this.balancerStrategy = balancerStrategy; + this.broadcastDatasources = broadcastDatasources; } public DruidCoordinatorRuntimeParams build() @@ -324,7 +340,8 @@ public DruidCoordinatorRuntimeParams build() coordinatorCompactionConfig, stats, balancerReferenceTimestamp, - balancerStrategy + balancerStrategy, + broadcastDatasources ); } @@ -436,5 +453,11 @@ public Builder withBalancerStrategy(BalancerStrategy balancerStrategy) this.balancerStrategy = balancerStrategy; return this; } + + public Builder withBroadcastDatasources(Set broadcastDatasources) + { + this.broadcastDatasources = broadcastDatasources; + return this; + } } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java index 8b0b30698175..72fdedf6e453 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/RandomBalancerStrategy.java @@ -26,6 +26,7 @@ import java.util.Iterator; import java.util.List; import java.util.NavigableSet; +import java.util.Set; import java.util.concurrent.ThreadLocalRandom; public class RandomBalancerStrategy implements BalancerStrategy @@ -51,9 +52,9 @@ public ServerHolder findNewSegmentHomeBalancer(DataSegment proposalSegment, List } @Override - public BalancerSegmentHolder pickSegmentToMove(List serverHolders) + public BalancerSegmentHolder pickSegmentToMove(List serverHolders, Set broadcastDatasources) { - return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders); + return ReservoirSegmentSampler.getRandomBalancerSegmentHolder(serverHolders, broadcastDatasources); } @Override diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java index c2c4a7ad10c4..7181d52e152a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java @@ -22,19 +22,33 @@ import org.apache.druid.timeline.DataSegment; import java.util.List; +import java.util.Set; import java.util.concurrent.ThreadLocalRandom; final class ReservoirSegmentSampler { - static BalancerSegmentHolder getRandomBalancerSegmentHolder(final List serverHolders) + static BalancerSegmentHolder getRandomBalancerSegmentHolder( + final List serverHolders, + Set broadcastDatasources + ) { ServerHolder fromServerHolder = null; DataSegment proposalSegment = null; int numSoFar = 0; for (ServerHolder server : serverHolders) { + if (!server.getServer().getType().isSegmentReplicationTarget()) { + // if the server only handles broadcast segments (which don't need to be rebalanced), we have nothing to do + continue; + } + for (DataSegment segment : server.getServer().iterateAllSegments()) { + if (broadcastDatasources.contains(segment.getDataSource())) { + // we don't need to rebalance segments that were assigned via broadcast rules + continue; + } + int randNum = ThreadLocalRandom.current().nextInt(numSoFar + 1); // w.p. 1 / (numSoFar+1), swap out the server and segment if (randNum == numSoFar) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java b/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java index cd04bfdcb8a5..b86ca0106d39 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java @@ -21,6 +21,8 @@ import com.google.common.collect.HashBasedTable; import com.google.common.collect.Table; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -62,19 +64,22 @@ public static SegmentReplicantLookup make(DruidCluster cluster) } } - return new SegmentReplicantLookup(segmentsInCluster, loadingSegments); + return new SegmentReplicantLookup(segmentsInCluster, loadingSegments, cluster); } private final Table segmentsInCluster; private final Table loadingSegments; + private final DruidCluster cluster; private SegmentReplicantLookup( Table segmentsInCluster, - Table loadingSegments + Table loadingSegments, + DruidCluster cluster ) { this.segmentsInCluster = segmentsInCluster; this.loadingSegments = loadingSegments; + this.cluster = cluster; } public Map getClusterTiers(SegmentId segmentId) @@ -93,7 +98,7 @@ int getLoadedReplicants(SegmentId segmentId) return retVal; } - int getLoadedReplicants(SegmentId segmentId, String tier) + public int getLoadedReplicants(SegmentId segmentId, String tier) { Integer retVal = segmentsInCluster.get(segmentId, tier); return (retVal == null) ? 0 : retVal; @@ -124,4 +129,21 @@ public int getTotalReplicants(SegmentId segmentId, String tier) { return getLoadedReplicants(segmentId, tier) + getLoadingReplicants(segmentId, tier); } + + public Object2LongMap getBroadcastUnderReplication(SegmentId segmentId) + { + Object2LongOpenHashMap perTier = new Object2LongOpenHashMap<>(); + for (ServerHolder holder : cluster.getAllServers()) { + // Only record tier entry for server that is segment broadcast target + if (holder.getServer().getType().isSegmentBroadcastTarget()) { + // Every broadcast target server should be serving 1 replica of the segment + if (!holder.isServingSegment(segmentId)) { + perTier.addTo(holder.getServer().getTier(), 1L); + } else { + perTier.putIfAbsent(holder.getServer().getTier(), 0); + } + } + } + return perTier; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java index ba96566a4dfd..43fdaaef1d1d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ServerHolder.java @@ -22,6 +22,7 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; import java.util.Objects; @@ -114,7 +115,7 @@ public long getAvailableSize() public boolean isServingSegment(DataSegment segment) { - return server.getSegment(segment.getId()) != null; + return isServingSegment(segment.getId()); } public boolean isLoadingSegment(DataSegment segment) @@ -122,11 +123,21 @@ public boolean isLoadingSegment(DataSegment segment) return peon.getSegmentsToLoad().contains(segment); } + public boolean isDroppingSegment(DataSegment segment) + { + return peon.getSegmentsToDrop().contains(segment); + } + public int getNumberOfSegmentsInQueue() { return peon.getNumberOfSegmentsInQueue(); } + public boolean isServingSegment(SegmentId segmentId) + { + return server.getSegment(segmentId) != null; + } + @Override public int compareTo(ServerHolder serverHolder) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java index d42ca635bd62..a1c5237ddd15 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/BalanceSegments.java @@ -187,7 +187,10 @@ private Pair balanceServers( //noinspection ForLoopThatDoesntUseLoopVariable for (int iter = 0; (moved + unmoved) < maxSegmentsToMove; ++iter) { - final BalancerSegmentHolder segmentToMoveHolder = strategy.pickSegmentToMove(toMoveFrom); + final BalancerSegmentHolder segmentToMoveHolder = strategy.pickSegmentToMove( + toMoveFrom, + params.getBroadcastDatasources() + ); if (segmentToMoveHolder == null) { log.info("All servers to move segments from are empty, ending run."); break; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegments.java index febf9a4deafd..e278a7582302 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegments.java @@ -58,19 +58,17 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { - ImmutableDruidServer server = serverHolder.getServer(); - - for (ImmutableDruidDataSource dataSource : server.getDataSources()) { - VersionedIntervalTimeline timeline = timelines - .computeIfAbsent( - dataSource.getName(), - dsName -> new VersionedIntervalTimeline<>(Comparator.naturalOrder()) - ); - VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); - } + addSegmentsFromServer(serverHolder, timelines); } } + for (ServerHolder serverHolder : cluster.getBrokers()) { + addSegmentsFromServer(serverHolder, timelines); + } + + // Note that we do not include segments from ingestion services such as tasks or indexers, + // to prevent unpublished segments from prematurely overshadowing segments. + // Mark all segments as unused in db that are overshadowed by served segments for (DataSegment dataSegment : params.getUsedSegments()) { VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); @@ -83,4 +81,21 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params.buildFromExisting().withCoordinatorStats(stats).build(); } + + private void addSegmentsFromServer( + ServerHolder serverHolder, + Map> timelines + ) + { + ImmutableDruidServer server = serverHolder.getServer(); + + for (ImmutableDruidDataSource dataSource : server.getDataSources()) { + VersionedIntervalTimeline timeline = timelines + .computeIfAbsent( + dataSource.getName(), + dsName -> new VersionedIntervalTimeline<>(Comparator.naturalOrder()) + ); + VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); + } + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java index 5288bb35858f..d8c207148ffe 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/RunRules.java @@ -20,6 +20,7 @@ package org.apache.druid.server.coordinator.duty; import com.google.common.collect.Lists; +import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.MetadataRuleManager; @@ -28,11 +29,13 @@ import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.ReplicationThrottler; +import org.apache.druid.server.coordinator.rules.BroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; +import java.util.HashSet; import java.util.List; import java.util.Set; @@ -101,6 +104,21 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); int missingRules = 0; + + final Set broadcastDatasources = new HashSet<>(); + for (ImmutableDruidDataSource dataSource : params.getDataSourcesSnapshot().getDataSourcesMap().values()) { + List rules = databaseRuleManager.getRulesWithDefault(dataSource.getName()); + for (Rule rule : rules) { + // A datasource is considered a broadcast datasource if it has any broadcast rules. + // The set of broadcast datasources is used by BalanceSegments, so it's important that RunRules + // executes before BalanceSegments. + if (rule instanceof BroadcastDistributionRule) { + broadcastDatasources.add(dataSource.getName()); + break; + } + } + } + for (DataSegment segment : params.getUsedSegments()) { if (overshadowed.contains(segment.getId())) { // Skipping overshadowed segments @@ -131,6 +149,9 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) .emit(); } - return params.buildFromExisting().withCoordinatorStats(stats).build(); + return params.buildFromExisting() + .withCoordinatorStats(stats) + .withBroadcastDatasources(broadcastDatasources) + .build(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java index bea7a9d1cc79..bd8b2c30d550 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegments.java @@ -27,13 +27,18 @@ import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.rules.BroadcastDistributionRule; +import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.timeline.DataSegment; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import java.util.Set; import java.util.SortedSet; /** - * Unloads segments that are no longer marked as used from Historical servers. + * Unloads segments that are no longer marked as used from servers. */ public class UnloadUnusedSegments implements CoordinatorDuty { @@ -46,31 +51,102 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) Set usedSegments = params.getUsedSegments(); DruidCluster cluster = params.getDruidCluster(); + Map broadcastStatusByDatasource = new HashMap<>(); + for (String broadcastDatasource : params.getBroadcastDatasources()) { + broadcastStatusByDatasource.put(broadcastDatasource, true); + } + for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { - ImmutableDruidServer server = serverHolder.getServer(); + handleUnusedSegmentsForServer( + serverHolder, + usedSegments, + params, + stats, + false, + broadcastStatusByDatasource + ); + } + } - for (ImmutableDruidDataSource dataSource : server.getDataSources()) { - for (DataSegment segment : dataSource.getSegments()) { - if (!usedSegments.contains(segment)) { - LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName()); + for (ServerHolder serverHolder : cluster.getBrokers()) { + handleUnusedSegmentsForServer( + serverHolder, + usedSegments, + params, + stats, + false, + broadcastStatusByDatasource + ); + } + + for (ServerHolder serverHolder : cluster.getRealtimes()) { + handleUnusedSegmentsForServer( + serverHolder, + usedSegments, + params, + stats, + true, + broadcastStatusByDatasource + ); + } + + return params.buildFromExisting().withCoordinatorStats(stats).build(); + } - if (!queuePeon.getSegmentsToDrop().contains(segment)) { - queuePeon.dropSegment(segment, () -> {}); - stats.addToTieredStat("unneededCount", server.getTier(), 1); - log.info( - "Dropping uneeded segment [%s] from server [%s] in tier [%s]", - segment.getId(), - server.getName(), - server.getTier() - ); + private void handleUnusedSegmentsForServer( + ServerHolder serverHolder, + Set usedSegments, + DruidCoordinatorRuntimeParams params, + CoordinatorStats stats, + boolean dropBroadcastOnly, + Map broadcastStatusByDatasource + ) + { + ImmutableDruidServer server = serverHolder.getServer(); + for (ImmutableDruidDataSource dataSource : server.getDataSources()) { + boolean isBroadcastDatasource = broadcastStatusByDatasource.computeIfAbsent( + dataSource.getName(), + (dataSourceName) -> { + List rules = params.getDatabaseRuleManager().getRulesWithDefault(dataSource.getName()); + for (Rule rule : rules) { + // A datasource is considered a broadcast datasource if it has any broadcast rules. + if (rule instanceof BroadcastDistributionRule) { + return true; } } + return false; + } + ); + + // The coordinator tracks used segments by examining the metadata store. + // For tasks, the segments they create are unpublished, so those segments will get dropped + // unless we exclude them here. We currently drop only broadcast segments in that case. + // This check relies on the assumption that queryable stream tasks will never + // ingest data to a broadcast datasource. If a broadcast datasource is switched to become a non-broadcast + // datasource, this will result in the those segments not being dropped from tasks. + // A more robust solution which requires a larger rework could be to expose + // the set of segments that were created by a task/indexer here, and exclude them. + if (dropBroadcastOnly && !isBroadcastDatasource) { + continue; + } + + for (DataSegment segment : dataSource.getSegments()) { + if (!usedSegments.contains(segment)) { + LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName()); + + if (!queuePeon.getSegmentsToDrop().contains(segment)) { + queuePeon.dropSegment(segment, () -> {}); + stats.addToTieredStat("unneededCount", server.getTier(), 1); + log.info( + "Dropping uneeded segment [%s] from server [%s] in tier [%s]", + segment.getId(), + server.getName(), + server.getTier() + ); } } } } - - return params.buildFromExisting().withCoordinatorStats(stats).build(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java index 658171236adc..0b8c37b79318 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRule.java @@ -19,16 +19,21 @@ package org.apache.druid.server.coordinator.rules; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.SegmentReplicantLookup; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; import java.util.HashSet; -import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; public abstract class BroadcastDistributionRule implements Rule { @@ -37,34 +42,70 @@ public abstract class BroadcastDistributionRule implements Rule @Override public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) { - // Find servers which holds the segments of co-located data source - final Set loadServerHolders = new HashSet<>(); final Set dropServerHolders = new HashSet<>(); - final List colocatedDataSources = getColocatedDataSources(); - if (colocatedDataSources == null || colocatedDataSources.isEmpty()) { - loadServerHolders.addAll(params.getDruidCluster().getAllServers()); - } else { - params.getDruidCluster().getAllServers().forEach( - eachHolder -> { - if (!eachHolder.isDecommissioning() - && colocatedDataSources.stream() - .anyMatch(source -> eachHolder.getServer().getDataSource(source) != null)) { - loadServerHolders.add(eachHolder); - } else if (eachHolder.isServingSegment(segment)) { - if (!eachHolder.getPeon().getSegmentsToDrop().contains(segment)) { - dropServerHolders.add(eachHolder); - } - } - } - ); - } - final CoordinatorStats stats = new CoordinatorStats(); + // Find servers where we need to load the broadcast segments + final Set loadServerHolders = + params.getDruidCluster().getAllServers() + .stream() + .filter( + (serverHolder) -> { + ServerType serverType = serverHolder.getServer().getType(); + if (!serverType.isSegmentBroadcastTarget()) { + return false; + } + + final boolean isServingSegment = + serverHolder.isServingSegment(segment); + + if (serverHolder.isDecommissioning()) { + if (isServingSegment && !serverHolder.isDroppingSegment(segment)) { + dropServerHolders.add(serverHolder); + } + return false; + } + + return !isServingSegment && !serverHolder.isLoadingSegment(segment); + } + ) + .collect(Collectors.toSet()); + final CoordinatorStats stats = new CoordinatorStats(); return stats.accumulate(assign(loadServerHolders, segment)) .accumulate(drop(dropServerHolders, segment)); } + @Override + public boolean canLoadSegments() + { + return true; + } + + @Override + public void updateUnderReplicated( + Map> underReplicatedPerTier, + SegmentReplicantLookup segmentReplicantLookup, + DataSegment segment + ) + { + Object2LongMap underReplicatedBroadcastTiers = segmentReplicantLookup.getBroadcastUnderReplication(segment.getId()); + for (final Object2LongMap.Entry entry : underReplicatedBroadcastTiers.object2LongEntrySet()) { + final String tier = entry.getKey(); + final long underReplicatedCount = entry.getLongValue(); + underReplicatedPerTier.compute(tier, (_tier, existing) -> { + Object2LongMap underReplicationPerDataSource = existing; + if (existing == null) { + underReplicationPerDataSource = new Object2LongOpenHashMap<>(); + } + underReplicationPerDataSource.compute( + segment.getDataSource(), + (_datasource, count) -> count != null ? count + underReplicatedCount : underReplicatedCount + ); + return underReplicationPerDataSource; + }); + } + } + private CoordinatorStats assign( final Set serverHolders, final DataSegment segment @@ -110,6 +151,4 @@ private CoordinatorStats drop( return stats; } - - public abstract List getColocatedDataSources(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java index c565df9b58be..7ffc7a211551 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java @@ -37,4 +37,10 @@ public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntim stats.addToGlobalStat("deletedCount", 1); return stats; } + + @Override + public boolean canLoadSegments() + { + return false; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java index d095f1100aea..ef5094cbea4a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/ForeverBroadcastDistributionRule.java @@ -25,21 +25,16 @@ import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.List; import java.util.Objects; public class ForeverBroadcastDistributionRule extends BroadcastDistributionRule { static final String TYPE = "broadcastForever"; - private final List colocatedDataSources; - @JsonCreator - public ForeverBroadcastDistributionRule( - @JsonProperty("colocatedDataSources") List colocatedDataSources - ) + public ForeverBroadcastDistributionRule() { - this.colocatedDataSources = colocatedDataSources; + } @Override @@ -49,13 +44,6 @@ public String getType() return TYPE; } - @Override - @JsonProperty - public List getColocatedDataSources() - { - return colocatedDataSources; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -79,13 +67,12 @@ public boolean equals(Object o) return false; } - ForeverBroadcastDistributionRule that = (ForeverBroadcastDistributionRule) o; - return Objects.equals(colocatedDataSources, that.colocatedDataSources); + return true; } @Override public int hashCode() { - return Objects.hash(getType(), colocatedDataSources); + return Objects.hash(getType()); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java index c40dff7268aa..b1bf29eedd20 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/IntervalBroadcastDistributionRule.java @@ -25,23 +25,19 @@ import org.joda.time.DateTime; import org.joda.time.Interval; -import java.util.List; import java.util.Objects; public class IntervalBroadcastDistributionRule extends BroadcastDistributionRule { static final String TYPE = "broadcastByInterval"; private final Interval interval; - private final List colocatedDataSources; @JsonCreator public IntervalBroadcastDistributionRule( - @JsonProperty("interval") Interval interval, - @JsonProperty("colocatedDataSources") List colocatedDataSources + @JsonProperty("interval") Interval interval ) { this.interval = interval; - this.colocatedDataSources = colocatedDataSources; } @Override @@ -51,13 +47,6 @@ public String getType() return TYPE; } - @Override - @JsonProperty - public List getColocatedDataSources() - { - return colocatedDataSources; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -79,26 +68,19 @@ public Interval getInterval() @Override public boolean equals(Object o) { - if (o == this) { + if (this == o) { return true; } - - if (o == null || o.getClass() != getClass()) { + if (o == null || getClass() != o.getClass()) { return false; } - IntervalBroadcastDistributionRule that = (IntervalBroadcastDistributionRule) o; - - if (!Objects.equals(interval, that.interval)) { - return false; - } - - return Objects.equals(colocatedDataSources, that.colocatedDataSources); + return Objects.equals(getInterval(), that.getInterval()); } @Override public int hashCode() { - return Objects.hash(getType(), interval, colocatedDataSources); + return Objects.hash(getInterval()); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java index 80912544a4e9..d0bf5b8ad56b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/LoadRule.java @@ -21,6 +21,8 @@ import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -30,6 +32,7 @@ import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.ReplicationThrottler; +import org.apache.druid.server.coordinator.SegmentReplicantLookup; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -90,6 +93,32 @@ public CoordinatorStats run( } } + @Override + public boolean canLoadSegments() + { + return true; + } + + @Override + public void updateUnderReplicated( + Map> underReplicatedPerTier, + SegmentReplicantLookup segmentReplicantLookup, + DataSegment segment + ) + { + getTieredReplicants().forEach((final String tier, final Integer ruleReplicants) -> { + int currentReplicants = segmentReplicantLookup.getLoadedReplicants(segment.getId(), tier); + Object2LongMap underReplicationPerDataSource = underReplicatedPerTier.computeIfAbsent( + tier, + ignored -> new Object2LongOpenHashMap<>() + ); + ((Object2LongOpenHashMap) underReplicationPerDataSource).addTo( + segment.getDataSource(), + Math.max(ruleReplicants - currentReplicants, 0) + ); + }); + } + /** * @param stats {@link CoordinatorStats} to accumulate assignment statistics. */ diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java index 97c6e11cfba2..d48353d3e50a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/PeriodBroadcastDistributionRule.java @@ -26,7 +26,6 @@ import org.joda.time.Interval; import org.joda.time.Period; -import java.util.List; import java.util.Objects; public class PeriodBroadcastDistributionRule extends BroadcastDistributionRule @@ -36,18 +35,15 @@ public class PeriodBroadcastDistributionRule extends BroadcastDistributionRule private final Period period; private final boolean includeFuture; - private final List colocatedDataSources; @JsonCreator public PeriodBroadcastDistributionRule( @JsonProperty("period") Period period, - @JsonProperty("includeFuture") Boolean includeFuture, - @JsonProperty("colocatedDataSources") List colocatedDataSources + @JsonProperty("includeFuture") Boolean includeFuture ) { this.period = period; this.includeFuture = includeFuture == null ? DEFAULT_INCLUDE_FUTURE : includeFuture; - this.colocatedDataSources = colocatedDataSources; } @Override @@ -57,13 +53,6 @@ public String getType() return TYPE; } - @Override - @JsonProperty - public List getColocatedDataSources() - { - return colocatedDataSources; - } - @Override public boolean appliesTo(DataSegment segment, DateTime referenceTimestamp) { @@ -94,25 +83,17 @@ public boolean equals(Object o) if (this == o) { return true; } - - if (o == null || o.getClass() != getClass()) { + if (o == null || getClass() != o.getClass()) { return false; } - PeriodBroadcastDistributionRule that = (PeriodBroadcastDistributionRule) o; - - if (!Objects.equals(period, that.period)) { - return false; - } - if (includeFuture != that.includeFuture) { - return false; - } - return Objects.equals(colocatedDataSources, that.colocatedDataSources); + return isIncludeFuture() == that.isIncludeFuture() && + Objects.equals(getPeriod(), that.getPeriod()); } @Override public int hashCode() { - return Objects.hash(getType(), period, colocatedDataSources); + return Objects.hash(getPeriod(), isIncludeFuture()); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java index d475f67d4c1b..02c552f639cc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java @@ -21,13 +21,18 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.base.Preconditions; +import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.SegmentReplicantLookup; import org.apache.druid.timeline.DataSegment; import org.joda.time.DateTime; import org.joda.time.Interval; +import java.util.Map; + /** */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @@ -51,6 +56,27 @@ public interface Rule boolean appliesTo(Interval interval, DateTime referenceTimestamp); + /** + * Return true if this Rule can load segment onto one or more type of Druid node, otherwise return false. + * Any Rule that returns true for this method should implement logic for calculating segment under replicated + * in {@link Rule#updateUnderReplicated} + */ + boolean canLoadSegments(); + + /** + * This method should update the {@param underReplicatedPerTier} with the replication count of the + * {@param segment}. Rule that returns true for {@link Rule#canLoadSegments()} must override this method. + * Note that {@param underReplicatedPerTier} is a map of tier -> { dataSource -> underReplicationCount } + */ + default void updateUnderReplicated( + Map> underReplicatedPerTier, + SegmentReplicantLookup segmentReplicantLookup, + DataSegment segment + ) + { + Preconditions.checkArgument(!canLoadSegments()); + } + /** * {@link DruidCoordinatorRuntimeParams#getUsedSegments()} must not be called in Rule's code, because the used * segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is because diff --git a/server/src/main/java/org/apache/druid/server/emitter/ComposingEmitterModule.java b/server/src/main/java/org/apache/druid/server/emitter/ComposingEmitterModule.java index 27a35fd68df5..e466319a3b07 100644 --- a/server/src/main/java/org/apache/druid/server/emitter/ComposingEmitterModule.java +++ b/server/src/main/java/org/apache/druid/server/emitter/ComposingEmitterModule.java @@ -53,7 +53,7 @@ public void configure(Binder binder) @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } @Provides diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 040297521b80..f88d1c70854c 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -22,11 +22,13 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Optional; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; +import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.commons.lang.StringUtils; import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.DruidDataSource; @@ -49,6 +51,7 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.server.coordination.DruidServerMetadata; +import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.http.security.DatasourceResourceFilter; @@ -96,12 +99,14 @@ public class DataSourcesResource { private static final Logger log = new Logger(DataSourcesResource.class); + private static final long DEFAULT_LOADSTATUS_INTERVAL_OFFSET = 14 * 24 * 60 * 60 * 1000; private final CoordinatorServerView serverInventoryView; private final SegmentsMetadataManager segmentsMetadataManager; private final MetadataRuleManager metadataRuleManager; private final IndexingServiceClient indexingServiceClient; private final AuthorizerMapper authorizerMapper; + private final DruidCoordinator coordinator; @Inject public DataSourcesResource( @@ -109,7 +114,8 @@ public DataSourcesResource( SegmentsMetadataManager segmentsMetadataManager, MetadataRuleManager metadataRuleManager, @Nullable IndexingServiceClient indexingServiceClient, - AuthorizerMapper authorizerMapper + AuthorizerMapper authorizerMapper, + DruidCoordinator coordinator ) { this.serverInventoryView = serverInventoryView; @@ -117,6 +123,7 @@ public DataSourcesResource( this.metadataRuleManager = metadataRuleManager; this.indexingServiceClient = indexingServiceClient; this.authorizerMapper = authorizerMapper; + this.coordinator = coordinator; } @GET @@ -391,6 +398,130 @@ public Response getServedSegmentsInInterval( return getServedSegmentsInInterval(dataSourceName, full != null, theInterval::contains); } + @GET + @Path("/{dataSourceName}/loadstatus") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response getDatasourceLoadstatus( + @PathParam("dataSourceName") String dataSourceName, + @QueryParam("forceMetadataRefresh") final Boolean forceMetadataRefresh, + @QueryParam("interval") @Nullable final String interval, + @QueryParam("simple") @Nullable final String simple, + @QueryParam("full") @Nullable final String full + ) + { + if (forceMetadataRefresh == null) { + return Response + .status(Response.Status.BAD_REQUEST) + .entity("Invalid request. forceMetadataRefresh must be specified") + .build(); + } + final Interval theInterval; + if (interval == null) { + long currentTimeInMs = System.currentTimeMillis(); + theInterval = Intervals.utc(currentTimeInMs - DEFAULT_LOADSTATUS_INTERVAL_OFFSET, currentTimeInMs); + } else { + theInterval = Intervals.of(interval.replace('_', '/')); + } + + Optional> segments = segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval( + dataSourceName, + theInterval, + forceMetadataRefresh + ); + + if (!segments.isPresent()) { + return logAndCreateDataSourceNotFoundResponse(dataSourceName); + } + + if (Iterables.size(segments.get()) == 0) { + return Response + .status(Response.Status.NO_CONTENT) + .entity("No used segment found for the given datasource and interval") + .build(); + } + + if (simple != null) { + // Calculate response for simple mode + SegmentsLoadStatistics segmentsLoadStatistics = computeSegmentLoadStatistics(segments.get()); + return Response.ok( + ImmutableMap.of( + dataSourceName, + segmentsLoadStatistics.getNumUnavailableSegments() + ) + ).build(); + } else if (full != null) { + // Calculate response for full mode + Map> segmentLoadMap + = coordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegments(segments.get()); + if (segmentLoadMap.isEmpty()) { + return Response.serverError() + .entity("Coordinator segment replicant lookup is not initialized yet. Try again later.") + .build(); + } + return Response.ok(segmentLoadMap).build(); + } else { + // Calculate response for default mode + SegmentsLoadStatistics segmentsLoadStatistics = computeSegmentLoadStatistics(segments.get()); + return Response.ok( + ImmutableMap.of( + dataSourceName, + 100 * ((double) (segmentsLoadStatistics.getNumLoadedSegments()) / (double) segmentsLoadStatistics.getNumPublishedSegments()) + ) + ).build(); + } + } + + private SegmentsLoadStatistics computeSegmentLoadStatistics(Iterable segments) + { + Map segmentLoadInfos = serverInventoryView.getSegmentLoadInfos(); + int numPublishedSegments = 0; + int numUnavailableSegments = 0; + int numLoadedSegments = 0; + for (DataSegment segment : segments) { + numPublishedSegments++; + if (!segmentLoadInfos.containsKey(segment.getId())) { + numUnavailableSegments++; + } else { + numLoadedSegments++; + } + } + return new SegmentsLoadStatistics(numPublishedSegments, numUnavailableSegments, numLoadedSegments); + } + + private static class SegmentsLoadStatistics + { + private int numPublishedSegments; + private int numUnavailableSegments; + private int numLoadedSegments; + + SegmentsLoadStatistics( + int numPublishedSegments, + int numUnavailableSegments, + int numLoadedSegments + ) + { + this.numPublishedSegments = numPublishedSegments; + this.numUnavailableSegments = numUnavailableSegments; + this.numLoadedSegments = numLoadedSegments; + } + + public int getNumPublishedSegments() + { + return numPublishedSegments; + } + + public int getNumUnavailableSegments() + { + return numUnavailableSegments; + } + + public int getNumLoadedSegments() + { + return numLoadedSegments; + } + } + /** * The property names belong to the public HTTP JSON API. */ @@ -769,7 +900,7 @@ static boolean isSegmentLoaded(Iterable servedSegments && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 && Iterables.any( - segmentLoadInfo.getServers(), DruidServerMetadata::segmentReplicatable + segmentLoadInfo.getServers(), DruidServerMetadata::isSegmentReplicationTarget )) { return true; } diff --git a/server/src/main/java/org/apache/druid/server/initialization/AuthenticatorMapperModule.java b/server/src/main/java/org/apache/druid/server/initialization/AuthenticatorMapperModule.java index d7139da3ec82..f2f2e990fe20 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/AuthenticatorMapperModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/AuthenticatorMapperModule.java @@ -62,11 +62,10 @@ public void configure(Binder binder) LifecycleModule.register(binder, AuthenticatorMapper.class); } - @SuppressWarnings("unchecked") @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } private static class AuthenticatorMapperProvider implements Provider diff --git a/server/src/main/java/org/apache/druid/server/initialization/AuthorizerMapperModule.java b/server/src/main/java/org/apache/druid/server/initialization/AuthorizerMapperModule.java index 48de9cbfc643..c8d9482b0450 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/AuthorizerMapperModule.java +++ b/server/src/main/java/org/apache/druid/server/initialization/AuthorizerMapperModule.java @@ -62,11 +62,10 @@ public void configure(Binder binder) LifecycleModule.register(binder, AuthorizerMapper.class); } - @SuppressWarnings("unchecked") @Override public List getJacksonModules() { - return Collections.EMPTY_LIST; + return Collections.emptyList(); } private static class AuthorizerMapperProvider implements Provider diff --git a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java index dd3961e04f42..30b93a18c84d 100644 --- a/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java +++ b/server/src/test/java/org/apache/druid/client/BrokerServerViewTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.smile.SmileFactory; import com.fasterxml.jackson.dataformat.smile.SmileGenerator; -import com.google.common.base.Function; import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -161,22 +160,15 @@ public void testMultipleServerAddedRemovedSegment() throws Exception final List druidServers = Lists.transform( ImmutableList.of("locahost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), - new Function() - { - @Override - public DruidServer apply(String input) - { - return new DruidServer( - input, - input, - null, - 10000000L, - ServerType.HISTORICAL, - "default_tier", - 0 - ); - } - } + input -> new DruidServer( + input, + input, + null, + 10000000L, + ServerType.HISTORICAL, + "default_tier", + 0 + ) ); for (DruidServer druidServer : druidServers) { @@ -190,14 +182,7 @@ public DruidServer apply(String input) Pair.of("2011-04-01/2011-04-09", "v2"), Pair.of("2011-04-06/2011-04-09", "v3"), Pair.of("2011-04-01/2011-04-02", "v3") - ), new Function, DataSegment>() - { - @Override - public DataSegment apply(Pair input) - { - return dataSegmentWithIntervalAndVersion(input.lhs, input.rhs); - } - } + ), input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) ); for (int i = 0; i < 5; ++i) { @@ -261,6 +246,114 @@ public DataSegment apply(Pair input) ); } + @Test + public void testMultipleServerAndBroker() throws Exception + { + segmentViewInitLatch = new CountDownLatch(1); + segmentAddedLatch = new CountDownLatch(6); + + // temporarily set latch count to 1 + segmentRemovedLatch = new CountDownLatch(1); + + setupViews(); + + final DruidServer druidBroker = new DruidServer( + "localhost:5", + "localhost:5", + null, + 10000000L, + ServerType.BROKER, + "default_tier", + 0 + ); + + final List druidServers = Lists.transform( + ImmutableList.of("locahost:0", "localhost:1", "localhost:2", "localhost:3", "localhost:4"), + input -> new DruidServer( + input, + input, + null, + 10000000L, + ServerType.HISTORICAL, + "default_tier", + 0 + ) + ); + + setupZNodeForServer(druidBroker, zkPathsConfig, jsonMapper); + for (DruidServer druidServer : druidServers) { + setupZNodeForServer(druidServer, zkPathsConfig, jsonMapper); + } + + final List segments = Lists.transform( + ImmutableList.of( + Pair.of("2011-04-01/2011-04-03", "v1"), + Pair.of("2011-04-03/2011-04-06", "v1"), + Pair.of("2011-04-01/2011-04-09", "v2"), + Pair.of("2011-04-06/2011-04-09", "v3"), + Pair.of("2011-04-01/2011-04-02", "v3") + ), + input -> dataSegmentWithIntervalAndVersion(input.lhs, input.rhs) + ); + + DataSegment brokerSegment = dataSegmentWithIntervalAndVersion("2011-04-01/2011-04-11", "v4"); + announceSegmentForServer(druidBroker, brokerSegment, zkPathsConfig, jsonMapper); + for (int i = 0; i < 5; ++i) { + announceSegmentForServer(druidServers.get(i), segments.get(i), zkPathsConfig, jsonMapper); + } + Assert.assertTrue(timing.forWaiting().awaitLatch(segmentViewInitLatch)); + Assert.assertTrue(timing.forWaiting().awaitLatch(segmentAddedLatch)); + + TimelineLookup timeline = brokerServerView.getTimeline( + DataSourceAnalysis.forDataSource(new TableDataSource("test_broker_server_view")) + ).get(); + + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + (List) timeline.lookup( + Intervals.of( + "2011-04-01/2011-04-09" + ) + ) + ); + + // unannounce the broker segment should do nothing to announcements + unannounceSegmentForServer(druidBroker, brokerSegment, zkPathsConfig); + Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); + + // renew segmentRemovedLatch since we still have 5 segments to unannounce + segmentRemovedLatch = new CountDownLatch(5); + + timeline = brokerServerView.getTimeline( + DataSourceAnalysis.forDataSource(new TableDataSource("test_broker_server_view")) + ).get(); + + // expect same set of segments as before + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "v3", druidServers.get(4), segments.get(4)), + createExpected("2011-04-02/2011-04-06", "v2", druidServers.get(2), segments.get(2)), + createExpected("2011-04-06/2011-04-09", "v3", druidServers.get(3), segments.get(3)) + ), + (List) timeline.lookup( + Intervals.of( + "2011-04-01/2011-04-09" + ) + ) + ); + + // unannounce all the segments + for (int i = 0; i < 5; ++i) { + unannounceSegmentForServer(druidServers.get(i), segments.get(i), zkPathsConfig); + } + Assert.assertTrue(timing.forWaiting().awaitLatch(segmentRemovedLatch)); + } + + private Pair>> createExpected( String intervalStr, String version, diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index ed6ececbeb86..30fb7659c11e 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -125,10 +125,10 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; +import org.apache.druid.timeline.partition.NumberedPartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.SingleDimensionShardSpec; import org.apache.druid.timeline.partition.SingleElementPartitionChunk; -import org.apache.druid.timeline.partition.StringPartitionChunk; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IAnswer; @@ -1486,19 +1486,19 @@ public void testSingleDimensionPruning() QueryRunner runner = new FinalizeResultsQueryRunner(getDefaultQueryRunner(), new TimeseriesQueryQueryToolChest()); final DruidServer lastServer = servers[random.nextInt(servers.length)]; - ServerSelector selector1 = makeMockSingleDimensionSelector(lastServer, "dim1", null, "b", 1); - ServerSelector selector2 = makeMockSingleDimensionSelector(lastServer, "dim1", "e", "f", 2); - ServerSelector selector3 = makeMockSingleDimensionSelector(lastServer, "dim1", "hi", "zzz", 3); - ServerSelector selector4 = makeMockSingleDimensionSelector(lastServer, "dim2", "a", "e", 4); - ServerSelector selector5 = makeMockSingleDimensionSelector(lastServer, "dim2", null, null, 5); - ServerSelector selector6 = makeMockSingleDimensionSelector(lastServer, "other", "b", null, 6); - - timeline.add(interval1, "v", new StringPartitionChunk<>(null, "a", 1, selector1)); - timeline.add(interval1, "v", new StringPartitionChunk<>("a", "b", 2, selector2)); - timeline.add(interval1, "v", new StringPartitionChunk<>("b", null, 3, selector3)); - timeline.add(interval2, "v", new StringPartitionChunk<>(null, "d", 4, selector4)); - timeline.add(interval2, "v", new StringPartitionChunk<>("d", null, 5, selector5)); - timeline.add(interval3, "v", new StringPartitionChunk<>(null, null, 6, selector6)); + ServerSelector selector1 = makeMockSingleDimensionSelector(lastServer, "dim1", null, "b", 0); + ServerSelector selector2 = makeMockSingleDimensionSelector(lastServer, "dim1", "e", "f", 1); + ServerSelector selector3 = makeMockSingleDimensionSelector(lastServer, "dim1", "hi", "zzz", 2); + ServerSelector selector4 = makeMockSingleDimensionSelector(lastServer, "dim2", "a", "e", 0); + ServerSelector selector5 = makeMockSingleDimensionSelector(lastServer, "dim2", null, null, 1); + ServerSelector selector6 = makeMockSingleDimensionSelector(lastServer, "other", "b", null, 0); + + timeline.add(interval1, "v", new NumberedPartitionChunk<>(0, 3, selector1)); + timeline.add(interval1, "v", new NumberedPartitionChunk<>(1, 3, selector2)); + timeline.add(interval1, "v", new NumberedPartitionChunk<>(2, 3, selector3)); + timeline.add(interval2, "v", new NumberedPartitionChunk<>(0, 2, selector4)); + timeline.add(interval2, "v", new NumberedPartitionChunk<>(1, 2, selector5)); + timeline.add(interval3, "v", new NumberedPartitionChunk<>(0, 1, selector6)); final Capture capture = Capture.newInstance(); final Capture contextCap = Capture.newInstance(); @@ -1514,10 +1514,10 @@ public void testSingleDimensionPruning() EasyMock.replay(mockRunner); List descriptors = new ArrayList<>(); - descriptors.add(new SegmentDescriptor(interval1, "v", 1)); - descriptors.add(new SegmentDescriptor(interval1, "v", 3)); - descriptors.add(new SegmentDescriptor(interval2, "v", 5)); - descriptors.add(new SegmentDescriptor(interval3, "v", 6)); + descriptors.add(new SegmentDescriptor(interval1, "v", 0)); + descriptors.add(new SegmentDescriptor(interval1, "v", 2)); + descriptors.add(new SegmentDescriptor(interval2, "v", 1)); + descriptors.add(new SegmentDescriptor(interval3, "v", 0)); MultipleSpecificSegmentSpec expected = new MultipleSpecificSegmentSpec(descriptors); runner.run(QueryPlus.wrap(query)).toList(); @@ -1538,7 +1538,13 @@ private ServerSelector makeMockSingleDimensionSelector( null, null, null, - new SingleDimensionShardSpec(dimension, start, end, partitionNum), + new SingleDimensionShardSpec( + dimension, + start, + end, + partitionNum, + SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS + ), null, 9, 0L @@ -1966,7 +1972,7 @@ private List> populateTimeline( final ShardSpec shardSpec; if (numChunks == 1) { - shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0); + shardSpec = new SingleDimensionShardSpec("dimAll", null, null, 0, 1); } else { String start = null; String end = null; @@ -1976,7 +1982,7 @@ private List> populateTimeline( if (j + 1 < numChunks) { end = String.valueOf(j + 1); } - shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j); + shardSpec = new SingleDimensionShardSpec("dim" + k, start, end, j, numChunks); } DataSegment mockSegment = makeMock(mocks, DataSegment.class); ServerExpectation expectation = new ServerExpectation<>( diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 5eb34ffbe28d..bd8e5ef3a8b0 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -1032,7 +1032,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce @Test public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IOException { - final PartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(null, 5); + final PartialShardSpec partialShardSpec = new HashBasedNumberedPartialShardSpec(null, 2, 5); final String dataSource = "ds"; final Interval interval = Intervals.of("2017-01-01/2017-02-01"); @@ -1048,7 +1048,8 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(0, shardSpec.getPartitionNum()); - Assert.assertEquals(5, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); + Assert.assertEquals(5, shardSpec.getNumBuckets()); coordinator.announceHistoricalSegments( Collections.singleton( @@ -1078,7 +1079,8 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(1, shardSpec.getPartitionNum()); - Assert.assertEquals(5, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); + Assert.assertEquals(5, shardSpec.getNumBuckets()); coordinator.announceHistoricalSegments( Collections.singleton( @@ -1101,13 +1103,14 @@ public void testAllocatePendingSegmentsForHashBasedNumberedShardSpec() throws IO "seq3", null, interval, - new HashBasedNumberedPartialShardSpec(null, 3), + new HashBasedNumberedPartialShardSpec(null, 2, 3), "version", true ); shardSpec = (HashBasedNumberedShardSpec) id.getShardSpec(); Assert.assertEquals(2, shardSpec.getPartitionNum()); - Assert.assertEquals(3, shardSpec.getPartitions()); + Assert.assertEquals(0, shardSpec.getNumCorePartitions()); + Assert.assertEquals(3, shardSpec.getNumBuckets()); } } diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java index be6354a63c75..57df44073460 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataManagerTest.java @@ -20,11 +20,13 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Optional; import com.google.common.base.Suppliers; 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 org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -43,6 +45,7 @@ import org.junit.Test; import java.io.IOException; +import java.util.Set; import java.util.stream.Collectors; @@ -117,7 +120,7 @@ public void setUp() throws Exception { TestDerbyConnector connector = derbyConnectorRule.getConnector(); SegmentsMetadataManagerConfig config = new SegmentsMetadataManagerConfig(); - config.setPollDuration(Period.seconds(1)); + config.setPollDuration(Period.seconds(3)); sqlSegmentsMetadataManager = new SqlSegmentsMetadataManager( jsonMapper, Suppliers.ofInstance(config), @@ -148,30 +151,124 @@ public void teardown() } @Test - public void testPoll() + public void testPollPeriodically() { + DataSourcesSnapshot dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertNull(dataSourcesSnapshot); sqlSegmentsMetadataManager.startPollingDatabasePeriodically(); - sqlSegmentsMetadataManager.poll(); Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); + // This call make sure that the first poll is completed + sqlSegmentsMetadataManager.useLatestSnapshotIfWithinDelay(); + Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.PeriodicDatabasePoll); + dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); Assert.assertEquals( ImmutableSet.of("wikipedia"), sqlSegmentsMetadataManager.retrieveAllDataSourceNames() ); Assert.assertEquals( ImmutableList.of("wikipedia"), - sqlSegmentsMetadataManager - .getImmutableDataSourcesWithAllUsedSegments() - .stream() - .map(ImmutableDruidDataSource::getName) - .collect(Collectors.toList()) + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(sqlSegmentsMetadataManager.getImmutableDataSourceWithUsedSegments("wikipedia").getSegments()) + ImmutableSet.copyOf(dataSourcesSnapshot.getDataSource("wikipedia").getSegments()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(sqlSegmentsMetadataManager.iterateAllUsedSegments()) + ImmutableSet.copyOf(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) + ); + } + + @Test + public void testPollOnDemand() + { + DataSourcesSnapshot dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertNull(dataSourcesSnapshot); + // This should return false and not wait/poll anything as we did not schedule periodic poll + Assert.assertFalse(sqlSegmentsMetadataManager.useLatestSnapshotIfWithinDelay()); + Assert.assertNull(dataSourcesSnapshot); + // This call will force on demand poll + sqlSegmentsMetadataManager.forceOrWaitOngoingDatabasePoll(); + Assert.assertFalse(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); + Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.OnDemandDatabasePoll); + dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertEquals( + ImmutableSet.of("wikipedia"), + sqlSegmentsMetadataManager.retrieveAllDataSourceNames() + ); + Assert.assertEquals( + ImmutableList.of("wikipedia"), + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) + ); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2), + ImmutableSet.copyOf(dataSourcesSnapshot.getDataSource("wikipedia").getSegments()) + ); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2), + ImmutableSet.copyOf(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) + ); + } + + @Test(timeout = 60_000) + public void testPollPeriodicallyAndOnDemandInterleave() throws Exception + { + DataSourcesSnapshot dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertNull(dataSourcesSnapshot); + sqlSegmentsMetadataManager.startPollingDatabasePeriodically(); + Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); + // This call make sure that the first poll is completed + sqlSegmentsMetadataManager.useLatestSnapshotIfWithinDelay(); + Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.PeriodicDatabasePoll); + dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertEquals( + ImmutableList.of("wikipedia"), + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) + ); + final String newDataSource2 = "wikipedia2"; + final DataSegment newSegment2 = createNewSegment1(newDataSource2); + publisher.publishSegment(newSegment2); + + // This call will force on demand poll + sqlSegmentsMetadataManager.forceOrWaitOngoingDatabasePoll(); + Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); + Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.OnDemandDatabasePoll); + // New datasource should now be in the snapshot since we just force on demand poll. + dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertEquals( + ImmutableList.of("wikipedia2", "wikipedia"), + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) + ); + + final String newDataSource3 = "wikipedia3"; + final DataSegment newSegment3 = createNewSegment1(newDataSource3); + publisher.publishSegment(newSegment3); + + // This time wait for periodic poll (not doing on demand poll so we have to wait a bit...) + while (sqlSegmentsMetadataManager.getDataSourcesSnapshot().getDataSource(newDataSource3) == null) { + Thread.sleep(1000); + } + Assert.assertTrue(sqlSegmentsMetadataManager.isPollingDatabasePeriodically()); + Assert.assertTrue(sqlSegmentsMetadataManager.getLatestDatabasePoll() instanceof SqlSegmentsMetadataManager.PeriodicDatabasePoll); + dataSourcesSnapshot = sqlSegmentsMetadataManager.getDataSourcesSnapshot(); + Assert.assertEquals( + ImmutableList.of("wikipedia2", "wikipedia3", "wikipedia"), + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) ); } @@ -749,4 +846,46 @@ public void testStopAndStart() sqlSegmentsMetadataManager.startPollingDatabasePeriodically(); sqlSegmentsMetadataManager.stopPollingDatabasePeriodically(); } + + @Test + public void testIterateAllUsedNonOvershadowedSegmentsForDatasourceInterval() throws Exception + { + final Interval theInterval = Intervals.of("2012-03-15T00:00:00.000/2012-03-20T00:00:00.000"); + Optional> segments = sqlSegmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval( + "wikipedia", theInterval, true + ); + Assert.assertTrue(segments.isPresent()); + Set dataSegmentSet = ImmutableSet.copyOf(segments.get()); + Assert.assertEquals(1, dataSegmentSet.size()); + Assert.assertTrue(dataSegmentSet.contains(segment1)); + + final DataSegment newSegment2 = createSegment( + "wikipedia", + "2012-03-16T00:00:00.000/2012-03-17T00:00:00.000", + "2017-10-15T20:19:12.565Z", + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 + ); + publisher.publishSegment(newSegment2); + + // New segment is not returned since we call without force poll + segments = sqlSegmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval( + "wikipedia", theInterval, false + ); + Assert.assertTrue(segments.isPresent()); + dataSegmentSet = ImmutableSet.copyOf(segments.get()); + Assert.assertEquals(1, dataSegmentSet.size()); + Assert.assertTrue(dataSegmentSet.contains(segment1)); + + // New segment is returned since we call with force poll + segments = sqlSegmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval( + "wikipedia", theInterval, true + ); + Assert.assertTrue(segments.isPresent()); + dataSegmentSet = ImmutableSet.copyOf(segments.get()); + Assert.assertEquals(2, dataSegmentSet.size()); + Assert.assertTrue(dataSegmentSet.contains(segment1)); + Assert.assertTrue(dataSegmentSet.contains(newSegment2)); + } + } 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 new file mode 100644 index 000000000000..67126b0c7b2f --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/InputSourceModuleTest.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.metadata.input; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.cfg.MapperConfig; +import com.fasterxml.jackson.databind.introspect.AnnotatedClass; +import com.fasterxml.jackson.databind.introspect.AnnotatedClassResolver; +import com.fasterxml.jackson.databind.jsontype.NamedType; +import com.google.common.collect.Iterables; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; +import java.util.stream.Collectors; + +public class InputSourceModuleTest +{ + private final ObjectMapper mapper = new ObjectMapper(); + private final String SQL_NAMED_TYPE = "sql"; + + @Before + public void setUp() + { + for (Module jacksonModule : new InputSourceModule().getJacksonModules()) { + mapper.registerModule(jacksonModule); + } + } + + @Test + public void testSubTypeRegistration() + { + MapperConfig config = mapper.getDeserializationConfig(); + AnnotatedClass annotatedClass = AnnotatedClassResolver.resolveWithoutSuperTypes(config, SqlInputSource.class); + List subtypes = mapper.getSubtypeResolver() + .collectAndResolveSubtypesByClass(config, annotatedClass) + .stream() + .map(NamedType::getName) + .collect(Collectors.toList()); + Assert.assertNotNull(subtypes); + Assert.assertEquals(SQL_NAMED_TYPE, Iterables.getOnlyElement(subtypes)); + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.java new file mode 100644 index 000000000000..46a171b50c92 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlEntityTest.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.metadata.input; + +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.commons.io.IOUtils; +import org.apache.druid.data.input.InputEntity; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; + +public class SqlEntityTest +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + + private final ObjectMapper mapper = TestHelper.makeSmileMapper(); + private TestDerbyConnector derbyConnector; + String TABLE_NAME_1 = "FOOS_TABLE"; + + String VALID_SQL = "SELECT timestamp,a,b FROM FOOS_TABLE"; + String INVALID_SQL = "DONT SELECT timestamp,a,b FROM FOOS_TABLE"; + String resultJson = "[{\"a\":\"0\"," + + "\"b\":\"0\"," + + "\"timestamp\":\"2011-01-12T00:00:00.000Z\"" + + "}]"; + + @Before + public void setUp() + { + for (Module jacksonModule : new InputSourceModule().getJacksonModules()) { + mapper.registerModule(jacksonModule); + } + } + + @Test + public void testExecuteQuery() throws IOException + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 1); + File tmpFile = File.createTempFile( + "testQueryResults", + "" + ); + InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( + VALID_SQL, + testUtils.getDerbyFirehoseConnector(), + mapper, + true, + tmpFile + ); + InputStream queryInputStream = new FileInputStream(queryResult.file()); + String actualJson = IOUtils.toString(queryInputStream, StandardCharsets.UTF_8); + + Assert.assertEquals(actualJson, resultJson); + testUtils.dropTable(TABLE_NAME_1); + } + + @Test(expected = IOException.class) + public void testFailOnInvalidQuery() throws IOException + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 1); + File tmpFile = File.createTempFile( + "testQueryResults", + "" + ); + InputEntity.CleanableFile queryResult = SqlEntity.openCleanableFile( + INVALID_SQL, + testUtils.getDerbyFirehoseConnector(), + mapper, + true, + tmpFile + ); + + Assert.assertTrue(tmpFile.exists()); + } + + @Test + public void testFileDeleteOnInvalidQuery() throws IOException + { + //The test parameters here are same as those used for testFailOnInvalidQuery(). + //The only difference is that this test checks if the temporary file is deleted upon failure. + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 1); + File tmpFile = File.createTempFile( + "testQueryResults", + "" + ); + try { + SqlEntity.openCleanableFile( + INVALID_SQL, + testUtils.getDerbyFirehoseConnector(), + mapper, + true, + tmpFile + ); + } + // Lets catch the exception so as to test temporary file deletion. + catch (IOException e) { + Assert.assertFalse(tmpFile.exists()); + } + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java new file mode 100644 index 000000000000..7afa88894f26 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlInputSourceTest.java @@ -0,0 +1,262 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.fasterxml.jackson.databind.Module; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.commons.dbcp2.BasicDataSource; +import org.apache.commons.io.FileUtils; +import org.apache.druid.data.input.InputFormat; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.data.input.InputRowSchema; +import org.apache.druid.data.input.InputSourceReader; +import org.apache.druid.data.input.InputSplit; +import org.apache.druid.data.input.Row; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.parsers.CloseableIterator; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.segment.TestHelper; +import org.easymock.EasyMock; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.skife.jdbi.v2.DBI; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class SqlInputSourceTest +{ + private static final List FIREHOSE_TMP_DIRS = new ArrayList<>(); + private final String TABLE_NAME_1 = "FOOS_TABLE_1"; + private final String TABLE_NAME_2 = "FOOS_TABLE_2"; + + private final List SQLLIST1 = ImmutableList.of("SELECT timestamp,a,b FROM FOOS_TABLE_1"); + private final List SQLLIST2 = ImmutableList.of( + "SELECT timestamp,a,b FROM FOOS_TABLE_1", + "SELECT timestamp,a,b FROM FOOS_TABLE_2" + ); + + private static final InputRowSchema INPUT_ROW_SCHEMA = new InputRowSchema( + new TimestampSpec("timestamp", "auto", null), + new DimensionsSpec( + DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "a", "b")), + new ArrayList<>(), + new ArrayList<>() + ), + Collections.emptyList() + ); + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private final ObjectMapper mapper = TestHelper.makeSmileMapper(); + private TestDerbyConnector derbyConnector; + + @Before + public void setUp() + { + for (Module jacksonModule : new InputSourceModule().getJacksonModules()) { + mapper.registerModule(jacksonModule); + } + } + + @AfterClass + public static void teardown() throws IOException + { + for (File dir : FIREHOSE_TMP_DIRS) { + FileUtils.forceDelete(dir); + } + } + + private void assertResult(List rows, List sqls) + { + Assert.assertEquals(10 * sqls.size(), rows.size()); + rows.sort(Comparator.comparing(Row::getTimestamp) + .thenComparingInt(r -> Integer.valueOf(r.getDimension("a").get(0))) + .thenComparingInt(r -> Integer.valueOf(r.getDimension("b").get(0)))); + int rowCount = 0; + for (int i = 0; i < 10; i++) { + for (int j = 0; j < sqls.size(); j++) { + final Row row = rows.get(rowCount); + String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); + Assert.assertEquals(timestampSt, row.getTimestamp().toString()); + Assert.assertEquals(i, Integer.valueOf(row.getDimension("a").get(0)).intValue()); + Assert.assertEquals(i, Integer.valueOf(row.getDimension("b").get(0)).intValue()); + rowCount++; + } + } + } + + private File createFirehoseTmpDir(String dirSuffix) throws IOException + { + final File firehoseTempDir = File.createTempFile( + SqlInputSourceTest.class.getSimpleName(), + dirSuffix + ); + FileUtils.forceDelete(firehoseTempDir); + FileUtils.forceMkdir(firehoseTempDir); + FIREHOSE_TMP_DIRS.add(firehoseTempDir); + return firehoseTempDir; + } + + @Test + public void testSerde() throws IOException + { + mapper.registerSubtypes(TestSerdeFirehoseConnector.class); + final SqlInputSourceTest.TestSerdeFirehoseConnector testSerdeFirehoseConnector = new SqlInputSourceTest.TestSerdeFirehoseConnector( + new MetadataStorageConnectorConfig()); + final SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST1, true, testSerdeFirehoseConnector, mapper); + final String valueString = mapper.writeValueAsString(sqlInputSource); + final SqlInputSource inputSourceFromJson = mapper.readValue(valueString, SqlInputSource.class); + Assert.assertEquals(sqlInputSource, inputSourceFromJson); + } + + @Test + public void testSingleSplit() throws Exception + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + final File tempDir = createFirehoseTmpDir("testSingleSplit"); + SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST1, true, testUtils.getDerbyFirehoseConnector(), mapper); + InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); + CloseableIterator resultIterator = sqlReader.read(); + final List rows = new ArrayList<>(); + while (resultIterator.hasNext()) { + rows.add(resultIterator.next()); + } + assertResult(rows, SQLLIST1); + testUtils.dropTable(TABLE_NAME_1); + } + + + @Test + public void testMultipleSplits() throws Exception + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + testUtils.createAndUpdateTable(TABLE_NAME_2, 10); + final File tempDir = createFirehoseTmpDir("testMultipleSplit"); + SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST2, true, testUtils.getDerbyFirehoseConnector(), mapper); + InputSourceReader sqlReader = sqlInputSource.fixedFormatReader(INPUT_ROW_SCHEMA, tempDir); + CloseableIterator resultIterator = sqlReader.read(); + final List rows = new ArrayList<>(); + while (resultIterator.hasNext()) { + rows.add(resultIterator.next()); + } + assertResult(rows, SQLLIST2); + testUtils.dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_2); + } + + @Test + public void testNumSplits() + { + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + SqlInputSource sqlInputSource = new SqlInputSource(SQLLIST2, true, testUtils.getDerbyFirehoseConnector(), mapper); + InputFormat inputFormat = EasyMock.createMock(InputFormat.class); + Stream> sqlSplits = sqlInputSource.createSplits(inputFormat, null); + Assert.assertEquals(SQLLIST2, sqlSplits.map(InputSplit::get).collect(Collectors.toList())); + Assert.assertEquals(2, sqlInputSource.estimateNumSplits(inputFormat, null)); + } + + @Test + public void testEquals() + { + EqualsVerifier.forClass(SqlInputSource.class) + .withPrefabValues( + ObjectMapper.class, + new ObjectMapper(), + new ObjectMapper() + ) + .withIgnoredFields("objectMapper") + .withNonnullFields("sqls", "sqlFirehoseDatabaseConnector") + .usingGetClass() + .verify(); + } + + @JsonTypeName("test") + private static class TestSerdeFirehoseConnector extends SQLFirehoseDatabaseConnector + { + private final DBI dbi; + private final MetadataStorageConnectorConfig metadataStorageConnectorConfig; + + private TestSerdeFirehoseConnector( + @JsonProperty("connectorConfig") MetadataStorageConnectorConfig metadataStorageConnectorConfig + ) + { + final BasicDataSource datasource = getDatasource(metadataStorageConnectorConfig); + datasource.setDriverClassLoader(getClass().getClassLoader()); + datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver"); + this.dbi = new DBI(datasource); + this.metadataStorageConnectorConfig = metadataStorageConnectorConfig; + } + + @JsonProperty("connectorConfig") + public MetadataStorageConnectorConfig getConnectorConfig() + { + return metadataStorageConnectorConfig; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TestSerdeFirehoseConnector that = (TestSerdeFirehoseConnector) o; + return metadataStorageConnectorConfig.equals(that.metadataStorageConnectorConfig); + } + + @Override + public int hashCode() + { + return Objects.hash(metadataStorageConnectorConfig); + } + + @Override + public DBI getDBI() + { + return dbi; + } + } +} diff --git a/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java new file mode 100644 index 000000000000..60e7c73e4397 --- /dev/null +++ b/server/src/test/java/org/apache/druid/metadata/input/SqlTestUtils.java @@ -0,0 +1,118 @@ +/* + * 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.metadata.input; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import org.apache.commons.dbcp2.BasicDataSource; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.metadata.MetadataStorageConnectorConfig; +import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; +import org.apache.druid.metadata.TestDerbyConnector; +import org.junit.Rule; +import org.skife.jdbi.v2.Batch; +import org.skife.jdbi.v2.DBI; +import org.skife.jdbi.v2.tweak.HandleCallback; + +public class SqlTestUtils +{ + @Rule + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private final TestDerbyFirehoseConnector derbyFirehoseConnector; + private final TestDerbyConnector derbyConnector; + + public SqlTestUtils(TestDerbyConnector derbyConnector) + { + this.derbyConnector = derbyConnector; + this.derbyFirehoseConnector = new SqlTestUtils.TestDerbyFirehoseConnector( + new MetadataStorageConnectorConfig(), + derbyConnector.getDBI() + ); + } + + private static class TestDerbyFirehoseConnector extends SQLFirehoseDatabaseConnector + { + private final DBI dbi; + + private TestDerbyFirehoseConnector( + @JsonProperty("connectorConfig") MetadataStorageConnectorConfig metadataStorageConnectorConfig, DBI dbi + ) + { + final BasicDataSource datasource = getDatasource(metadataStorageConnectorConfig); + datasource.setDriverClassLoader(getClass().getClassLoader()); + datasource.setDriverClassName("org.apache.derby.jdbc.ClientDriver"); + this.dbi = dbi; + } + + @Override + public DBI getDBI() + { + return dbi; + } + } + + public void createAndUpdateTable(final String tableName, int numEntries) + { + derbyConnector.createTable( + tableName, + ImmutableList.of( + StringUtils.format( + "CREATE TABLE %1$s (\n" + + " timestamp varchar(255) NOT NULL,\n" + + " a VARCHAR(255) NOT NULL,\n" + + " b VARCHAR(255) NOT NULL\n" + + ")", + tableName + ) + ) + ); + + derbyConnector.getDBI().withHandle( + (handle) -> { + Batch batch = handle.createBatch(); + for (int i = 0; i < numEntries; i++) { + String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); + batch.add(StringUtils.format("INSERT INTO %1$s (timestamp, a, b) VALUES ('%2$s', '%3$s', '%4$s')", + tableName, timestampSt, + i, i + )); + } + batch.execute(); + return null; + } + ); + } + + public void dropTable(final String tableName) + { + derbyConnector.getDBI().withHandle( + (HandleCallback) handle -> { + handle.createStatement(StringUtils.format("DROP TABLE %s", tableName)) + .execute(); + return null; + } + ); + } + + public TestDerbyFirehoseConnector getDerbyFirehoseConnector() + { + return derbyFirehoseConnector; + } +} diff --git a/server/src/test/java/org/apache/druid/query/lookup/LookupSnapshotTakerTest.java b/server/src/test/java/org/apache/druid/query/lookup/LookupSnapshotTakerTest.java index 8e1cbe9c8b17..a6e752ff65dc 100644 --- a/server/src/test/java/org/apache/druid/query/lookup/LookupSnapshotTakerTest.java +++ b/server/src/test/java/org/apache/druid/query/lookup/LookupSnapshotTakerTest.java @@ -125,7 +125,7 @@ public void tesLookupPullingFromEmptyFile() throws IOException { File snapshotFile = lookupSnapshotTaker.getPersistFile(TIER1); Assert.assertTrue(snapshotFile.createNewFile()); - Assert.assertEquals(Collections.EMPTY_LIST, lookupSnapshotTaker.pullExistingSnapshot(TIER1)); + Assert.assertEquals(Collections.emptyList(), lookupSnapshotTaker.pullExistingSnapshot(TIER1)); } @Test(expected = ISE.class) @@ -144,6 +144,6 @@ public void testLookupPullingFromNonExistingFile() throws IOException File directory = temporaryFolder.newFolder(); LookupSnapshotTaker lookupSnapshotTaker = new LookupSnapshotTaker(mapper, directory.getAbsolutePath()); List actualList = lookupSnapshotTaker.pullExistingSnapshot(TIER1); - Assert.assertEquals(Collections.EMPTY_LIST, actualList); + Assert.assertEquals(Collections.emptyList(), actualList); } } diff --git a/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java b/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java index d1be69840430..2a5bf3e5c7ce 100644 --- a/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/join/InlineJoinableFactoryTest.java @@ -80,6 +80,13 @@ public void testBuild() Assert.assertEquals(3, joinable.getCardinality("long")); } + @Test + public void testIsDirectlyJoinable() + { + Assert.assertTrue(factory.isDirectlyJoinable(inlineDataSource)); + Assert.assertFalse(factory.isDirectlyJoinable(new TableDataSource("foo"))); + } + private static JoinConditionAnalysis makeCondition(final String condition) { return JoinConditionAnalysis.forExpression(condition, PREFIX, ExprMacroTable.nil()); diff --git a/server/src/test/java/org/apache/druid/segment/join/LookupJoinableFactoryTest.java b/server/src/test/java/org/apache/druid/segment/join/LookupJoinableFactoryTest.java index 44ed4b2810af..6e0e737db9ed 100644 --- a/server/src/test/java/org/apache/druid/segment/join/LookupJoinableFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/join/LookupJoinableFactoryTest.java @@ -125,6 +125,13 @@ public void testBuild() Assert.assertEquals(Joinable.CARDINALITY_UNKNOWN, joinable.getCardinality("v")); } + @Test + public void testIsDirectlyJoinable() + { + Assert.assertTrue(factory.isDirectlyJoinable(lookupDataSource)); + Assert.assertFalse(factory.isDirectlyJoinable(new TableDataSource("foo"))); + } + private static JoinConditionAnalysis makeCondition(final String condition) { return JoinConditionAnalysis.forExpression(condition, PREFIX, ExprMacroTable.nil()); diff --git a/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java b/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java index e604b29f7631..a42c553bad5a 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java +++ b/server/src/test/java/org/apache/druid/segment/loading/CacheTestSegmentLoader.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.loading; import org.apache.druid.java.util.common.MapUtils; -import org.apache.druid.segment.AbstractSegment; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; @@ -49,7 +48,7 @@ public boolean isSegmentLoaded(DataSegment segment) @Override public Segment getSegment(final DataSegment segment, boolean lazy) { - return new AbstractSegment() + return new Segment() { @Override public SegmentId getId() 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 new file mode 100644 index 000000000000..4f288426e50f --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/FireHydrantTest.java @@ -0,0 +1,213 @@ +/* + * 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.realtime; + +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.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; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; +import java.util.Optional; +import java.util.function.Function; + +public class FireHydrantTest extends InitializedNullHandlingTest +{ + private IncrementalIndexSegment incrementalIndexSegment; + private QueryableIndexSegment queryableIndexSegment; + private FireHydrant hydrant; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Before + public void setup() + { + incrementalIndexSegment = new IncrementalIndexSegment(TestIndex.getIncrementalTestIndex(), SegmentId.dummy("test")); + queryableIndexSegment = new QueryableIndexSegment(TestIndex.getMMappedTestIndex(), SegmentId.dummy("test")); + + // hydrant starts out with incremental segment loaded + hydrant = new FireHydrant(incrementalIndexSegment, 0); + } + + @Test + public void testGetIncrementedSegmentNotSwapped() + { + Assert.assertEquals(0, hydrant.getHydrantSegment().getNumReferences()); + ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); + Assert.assertNotNull(segment); + Assert.assertTrue(segment.getBaseSegment() == incrementalIndexSegment); + Assert.assertEquals(1, segment.getNumReferences()); + } + + @Test + public void testGetIncrementedSegmentSwapped() + { + ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + hydrant.swapSegment(queryableIndexSegment); + ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); + Assert.assertNotNull(segment); + Assert.assertTrue(segment.getBaseSegment() == queryableIndexSegment); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + Assert.assertEquals(1, segment.getNumReferences()); + } + + @Test + public void testGetIncrementedSegmentClosed() + { + expectedException.expect(ISE.class); + expectedException.expectMessage("segment.close() is called somewhere outside FireHydrant.swapSegment()"); + hydrant.getHydrantSegment().close(); + Assert.assertEquals(0, hydrant.getHydrantSegment().getNumReferences()); + ReferenceCountingSegment segment = hydrant.getIncrementedSegment(); + } + + @Test + public void testGetAndIncrementSegment() throws IOException + { + ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + + Pair segmentAndCloseable = hydrant.getAndIncrementSegment(); + Assert.assertEquals(1, segmentAndCloseable.lhs.getNumReferences()); + segmentAndCloseable.rhs.close(); + Assert.assertEquals(0, segmentAndCloseable.lhs.getNumReferences()); + } + + @Test + public void testGetSegmentForQuery() throws IOException + { + ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + + Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Function.identity() + ); + Assert.assertTrue(maybeSegmentAndCloseable.isPresent()); + Assert.assertEquals(1, incrementalSegmentReference.getNumReferences()); + + Pair segmentAndCloseable = maybeSegmentAndCloseable.get(); + segmentAndCloseable.rhs.close(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + } + + @Test + public void testGetSegmentForQuerySwapped() throws IOException + { + ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + hydrant.swapSegment(queryableIndexSegment); + ReferenceCountingSegment queryableSegmentReference = hydrant.getHydrantSegment(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + Assert.assertEquals(0, queryableSegmentReference.getNumReferences()); + + Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Function.identity() + ); + Assert.assertTrue(maybeSegmentAndCloseable.isPresent()); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + Assert.assertEquals(1, queryableSegmentReference.getNumReferences()); + + Pair segmentAndCloseable = maybeSegmentAndCloseable.get(); + segmentAndCloseable.rhs.close(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + Assert.assertEquals(0, queryableSegmentReference.getNumReferences()); + } + + @Test + public void testGetSegmentForQueryButNotAbleToAcquireReferences() + { + ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + + Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + segmentReference -> new SegmentReference() + { + @Override + public Optional acquireReferences() + { + return Optional.empty(); + } + + @Override + public SegmentId getId() + { + return incrementalIndexSegment.getId(); + } + + @Override + public Interval getDataInterval() + { + return incrementalIndexSegment.getDataInterval(); + } + + @Nullable + @Override + public QueryableIndex asQueryableIndex() + { + return incrementalIndexSegment.asQueryableIndex(); + } + + @Override + public StorageAdapter asStorageAdapter() + { + return incrementalIndexSegment.asStorageAdapter(); + } + + @Override + public void close() + { + incrementalIndexSegment.close(); + } + } + ); + Assert.assertFalse(maybeSegmentAndCloseable.isPresent()); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + } + + @Test + public void testGetSegmentForQueryButNotAbleToAcquireReferencesSegmentClosed() + { + expectedException.expect(ISE.class); + expectedException.expectMessage("segment.close() is called somewhere outside FireHydrant.swapSegment()"); + ReferenceCountingSegment incrementalSegmentReference = hydrant.getHydrantSegment(); + Assert.assertEquals(0, incrementalSegmentReference.getNumReferences()); + incrementalSegmentReference.close(); + + Optional> maybeSegmentAndCloseable = hydrant.getSegmentForQuery( + Function.identity() + ); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java index 55ff5892fd76..558736980a85 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentIdWithShardSpecTest.java @@ -53,7 +53,7 @@ public void testSerde() throws Exception Assert.assertEquals(INTERVAL, id2.getInterval()); Assert.assertEquals(VERSION, id2.getVersion()); Assert.assertEquals(SHARD_SPEC_1.getPartitionNum(), id2.getShardSpec().getPartitionNum()); - Assert.assertEquals(SHARD_SPEC_1.getPartitions(), ((NumberedShardSpec) id2.getShardSpec()).getPartitions()); + Assert.assertEquals(SHARD_SPEC_1.getNumCorePartitions(), ((NumberedShardSpec) id2.getShardSpec()).getNumCorePartitions()); } @Test diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelperTest.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelperTest.java new file mode 100644 index 000000000000..1f2af7298809 --- /dev/null +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/SegmentPublisherHelperTest.java @@ -0,0 +1,173 @@ +/* + * 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.realtime.appenderator; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingNumberedShardSpec; +import org.apache.druid.timeline.partition.BuildingSingleDimensionShardSpec; +import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; +import org.apache.druid.timeline.partition.HashBucketShardSpec; +import org.apache.druid.timeline.partition.NumberedOverwriteShardSpec; +import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.ShardSpec; +import org.apache.druid.timeline.partition.SingleDimensionShardSpec; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Set; + +public class SegmentPublisherHelperTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Test + public void testAnnotateAtomicUpdateGroupSize() + { + final Set segments = ImmutableSet.of( + newSegment( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID, + 0, + 3, + (short) 1 + ) + ), + newSegment( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 1, + 0, + 3, + (short) 1 + ) + ), + newSegment( + new NumberedOverwriteShardSpec( + PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + 2, + 0, + 3, + (short) 1 + ) + ) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(NumberedOverwriteShardSpec.class, segment.getShardSpec().getClass()); + final NumberedOverwriteShardSpec shardSpec = (NumberedOverwriteShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getAtomicUpdateGroupSize()); + } + } + + @Test + public void testAnnotateCorePartitionSetSizeForNumberedShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new BuildingNumberedShardSpec(0)), + newSegment(new BuildingNumberedShardSpec(1)), + newSegment(new BuildingNumberedShardSpec(2)) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(NumberedShardSpec.class, segment.getShardSpec().getClass()); + final NumberedShardSpec shardSpec = (NumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); + } + } + + @Test + public void testAnnotateCorePartitionSetSizeForHashNumberedShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new BuildingHashBasedNumberedShardSpec(0, 0, 3, null, new ObjectMapper())), + newSegment(new BuildingHashBasedNumberedShardSpec(1, 1, 3, null, new ObjectMapper())), + newSegment(new BuildingHashBasedNumberedShardSpec(2, 2, 3, null, new ObjectMapper())) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass()); + final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); + } + } + + @Test + public void testAnnotateCorePartitionSetSizeForSingleDimensionShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new BuildingSingleDimensionShardSpec(0, "dim", null, "ccc", 0)), + newSegment(new BuildingSingleDimensionShardSpec(1, "dim", null, "ccc", 1)), + newSegment(new BuildingSingleDimensionShardSpec(2, "dim", null, "ccc", 2)) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + for (DataSegment segment : annotated) { + Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass()); + final SingleDimensionShardSpec shardSpec = (SingleDimensionShardSpec) segment.getShardSpec(); + Assert.assertEquals(3, shardSpec.getNumCorePartitions()); + } + } + + @Test + public void testAnnotateShardSpecDoNothing() + { + final Set segments = ImmutableSet.of( + newSegment(new NumberedShardSpec(0, 0)), + newSegment(new NumberedShardSpec(1, 0)), + newSegment(new NumberedShardSpec(2, 0)) + ); + final Set annotated = SegmentPublisherHelper.annotateShardSpec(segments); + Assert.assertEquals(segments, annotated); + } + + @Test + public void testAnnotateShardSpecThrowingExceptionForBucketNumberedShardSpec() + { + final Set segments = ImmutableSet.of( + newSegment(new HashBucketShardSpec(0, 3, null, new ObjectMapper())), + newSegment(new HashBucketShardSpec(1, 3, null, new ObjectMapper())), + newSegment(new HashBucketShardSpec(2, 3, null, new ObjectMapper())) + ); + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Cannot publish segments with shardSpec"); + SegmentPublisherHelper.annotateShardSpec(segments); + } + + private static DataSegment newSegment(ShardSpec shardSpec) + { + return new DataSegment( + "datasource", + Intervals.of("2020-01-01/P1d"), + "version", + null, + ImmutableList.of("dim"), + ImmutableList.of("met"), + shardSpec, + 9, + 10L + ); + } +} diff --git a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java index 34fa763613fe..189aa4984aac 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/firehose/SqlFirehoseFactoryTest.java @@ -34,6 +34,7 @@ import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.SQLFirehoseDatabaseConnector; import org.apache.druid.metadata.TestDerbyConnector; +import org.apache.druid.metadata.input.SqlTestUtils; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.transform.TransformSpec; import org.junit.AfterClass; @@ -41,9 +42,7 @@ import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; -import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.DBI; -import org.skife.jdbi.v2.tweak.HandleCallback; import java.io.File; import java.io.IOException; @@ -82,7 +81,6 @@ public class SqlFirehoseFactoryTest ) ); private TestDerbyConnector derbyConnector; - private TestDerbyFirehoseConnector derbyFirehoseConnector; @BeforeClass public static void setup() throws IOException @@ -139,56 +137,12 @@ private File createFirehoseTmpDir(String dirSuffix) throws IOException return firehoseTempDir; } - private void dropTable(final String tableName) - { - derbyConnector.getDBI().withHandle( - (HandleCallback) handle -> { - handle.createStatement(StringUtils.format("DROP TABLE %s", tableName)) - .execute(); - return null; - } - ); - } - - private void createAndUpdateTable(final String tableName) - { - derbyConnector = derbyConnectorRule.getConnector(); - derbyFirehoseConnector = new TestDerbyFirehoseConnector(new MetadataStorageConnectorConfig(), - derbyConnector.getDBI()); - derbyConnector.createTable( - tableName, - ImmutableList.of( - StringUtils.format( - "CREATE TABLE %1$s (\n" - + " timestamp varchar(255) NOT NULL,\n" - + " a VARCHAR(255) NOT NULL,\n" - + " b VARCHAR(255) NOT NULL\n" - + ")", - tableName - ) - ) - ); - - derbyConnector.getDBI().withHandle( - (handle) -> { - Batch batch = handle.createBatch(); - for (int i = 0; i < 10; i++) { - String timestampSt = StringUtils.format("2011-01-12T00:0%s:00.000Z", i); - batch.add(StringUtils.format("INSERT INTO %1$s (timestamp, a, b) VALUES ('%2$s', '%3$s', '%4$s')", - tableName, timestampSt, - i, i - )); - } - batch.execute(); - return null; - } - ); - } - @Test public void testWithoutCacheAndFetch() throws Exception { - createAndUpdateTable(TABLE_NAME_1); + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( SQLLIST1, @@ -197,7 +151,7 @@ public void testWithoutCacheAndFetch() throws Exception 0L, 0L, true, - derbyFirehoseConnector, + testUtils.getDerbyFirehoseConnector(), mapper ); @@ -211,14 +165,16 @@ public void testWithoutCacheAndFetch() throws Exception assertResult(rows, SQLLIST1); assertNumRemainingCacheFiles(firehoseTmpDir, 0); - dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_1); } @Test public void testWithoutCache() throws IOException { - createAndUpdateTable(TABLE_NAME_1); + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( SQLLIST1, @@ -227,7 +183,7 @@ public void testWithoutCache() throws IOException null, null, true, - derbyFirehoseConnector, + testUtils.getDerbyFirehoseConnector(), mapper ); @@ -242,15 +198,17 @@ public void testWithoutCache() throws IOException assertResult(rows, SQLLIST1); assertNumRemainingCacheFiles(firehoseTmpDir, 0); - dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_1); } @Test public void testWithCacheAndFetch() throws IOException { - createAndUpdateTable(TABLE_NAME_1); - createAndUpdateTable(TABLE_NAME_2); + derbyConnector = derbyConnectorRule.getConnector(); + SqlTestUtils testUtils = new SqlTestUtils(derbyConnector); + testUtils.createAndUpdateTable(TABLE_NAME_1, 10); + testUtils.createAndUpdateTable(TABLE_NAME_2, 10); final SqlFirehoseFactory factory = new SqlFirehoseFactory( @@ -260,7 +218,7 @@ public void testWithCacheAndFetch() throws IOException 0L, null, true, - derbyFirehoseConnector, + testUtils.getDerbyFirehoseConnector(), mapper ); @@ -274,8 +232,8 @@ public void testWithCacheAndFetch() throws IOException assertResult(rows, SQLLIST2); assertNumRemainingCacheFiles(firehoseTmpDir, 2); - dropTable(TABLE_NAME_1); - dropTable(TABLE_NAME_2); + testUtils.dropTable(TABLE_NAME_1); + testUtils.dropTable(TABLE_NAME_2); } private static class TestDerbyFirehoseConnector extends SQLFirehoseDatabaseConnector diff --git a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java b/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java index 47157347a413..f5534bd46a3b 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/plumber/CoordinatorBasedSegmentHandoffNotifierTest.java @@ -174,7 +174,7 @@ public void testHandoffChecksForAssignableServer() ) ); - Assert.assertFalse( + Assert.assertTrue( CoordinatorBasedSegmentHandoffNotifier.isHandOffComplete( Collections.singletonList( new ImmutableSegmentLoadInfo( 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 f224b5f580e4..565fee9fddd9 100644 --- a/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java +++ b/server/src/test/java/org/apache/druid/server/ClientQuerySegmentWalkerTest.java @@ -33,6 +33,7 @@ import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; +import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.JoinDataSource; import org.apache.druid.query.Query; @@ -70,7 +71,9 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.join.InlineJoinableFactory; +import org.apache.druid.segment.join.JoinConditionAnalysis; import org.apache.druid.segment.join.JoinType; +import org.apache.druid.segment.join.Joinable; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.server.initialization.ServerConfig; @@ -96,6 +99,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; /** * Tests ClientQuerySegmentWalker. @@ -112,6 +116,7 @@ public class ClientQuerySegmentWalkerTest private static final String FOO = "foo"; private static final String BAR = "bar"; private static final String MULTI = "multi"; + private static final String GLOBAL = "broadcast"; private static final Interval INTERVAL = Intervals.of("2000/P1Y"); private static final String VERSION = "A"; @@ -218,6 +223,40 @@ public void testTimeseriesOnTable() Assert.assertEquals(1, scheduler.getTotalReleased().get()); } + @Test + public void testTimeseriesOnAutomaticGlobalTable() + { + final TimeseriesQuery query = + Druids.newTimeseriesQueryBuilder() + .dataSource(GLOBAL) + .granularity(Granularities.ALL) + .intervals(Collections.singletonList(INTERVAL)) + .aggregators(new LongSumAggregatorFactory("sum", "n")) + .context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false)) + .build(); + + // expect global/joinable datasource to be automatically translated into a GlobalTableDataSource + final TimeseriesQuery expectedClusterQuery = + Druids.newTimeseriesQueryBuilder() + .dataSource(new GlobalTableDataSource(GLOBAL)) + .granularity(Granularities.ALL) + .intervals(Collections.singletonList(INTERVAL)) + .aggregators(new LongSumAggregatorFactory("sum", "n")) + .context(ImmutableMap.of(TimeseriesQuery.CTX_GRAND_TOTAL, false)) + .build(); + + testQuery( + query, + ImmutableList.of(ExpectedQuery.cluster(expectedClusterQuery)), + ImmutableList.of(new Object[]{INTERVAL.getStartMillis(), 10L}) + ); + + Assert.assertEquals(1, scheduler.getTotalRun().get()); + Assert.assertEquals(1, scheduler.getTotalPrioritizedAndLaned().get()); + Assert.assertEquals(1, scheduler.getTotalAcquired().get()); + Assert.assertEquals(1, scheduler.getTotalReleased().get()); + } + @Test public void testTimeseriesOnInline() { @@ -606,6 +645,20 @@ private void initWalker(final Map serverProperties, QuerySchedul final JoinableFactory joinableFactory = new MapJoinableFactory( ImmutableMap., JoinableFactory>builder() .put(InlineDataSource.class, new InlineJoinableFactory()) + .put(GlobalTableDataSource.class, new JoinableFactory() + { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return ((GlobalTableDataSource) dataSource).getName().equals(GLOBAL); + } + + @Override + public Optional build(DataSource dataSource, JoinConditionAnalysis condition) + { + return Optional.empty(); + } + }) .build() ); @@ -651,7 +704,8 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable> QueryToolChest getToolChest return conglomerate.findFactory(query).getToolchest(); } }, + joinableFactory, new RetryQueryRunnerConfig(), TestHelper.makeJsonMapper(), serverConfig, @@ -271,4 +280,26 @@ public int getNumMergeBuffers() return conglomerate; } + public static JoinableFactory makeJoinableFactoryForLookup( + LookupExtractorFactoryContainerProvider lookupProvider + ) + { + return makeJoinableFactoryFromDefault(lookupProvider, null); + } + + public static JoinableFactory makeJoinableFactoryFromDefault( + @Nullable LookupExtractorFactoryContainerProvider lookupProvider, + @Nullable Map, JoinableFactory> custom + ) + { + ImmutableMap.Builder, JoinableFactory> builder = ImmutableMap.builder(); + builder.put(InlineDataSource.class, new InlineJoinableFactory()); + if (lookupProvider != null) { + builder.put(LookupDataSource.class, new LookupJoinableFactory(lookupProvider)); + } + if (custom != null) { + builder.putAll(custom); + } + return MapJoinableFactoryTest.fromMap(builder.build()); + } } diff --git a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java index 04f796e9918d..a91411b8db73 100644 --- a/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/SegmentManagerTest.java @@ -26,7 +26,6 @@ import org.apache.druid.java.util.common.MapUtils; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.segment.AbstractSegment; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; @@ -52,6 +51,7 @@ import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -90,7 +90,7 @@ public void cleanup(DataSegment segment) } }; - private static class SegmentForTesting extends AbstractSegment + private static class SegmentForTesting implements Segment { private final String version; private final Interval interval; @@ -422,17 +422,19 @@ public void testLoadAndDropNonRootGenerationSegment() throws SegmentLoadingExcep @SuppressWarnings("RedundantThrows") // TODO remove when the bug in intelliJ is fixed. private void assertResult(List expectedExistingSegments) throws SegmentLoadingException { - final Map expectedDataSourceSizes = expectedExistingSegments - .stream() - .collect(Collectors.toMap(DataSegment::getDataSource, DataSegment::getSize, Long::sum)); - final Map expectedDataSourceCounts = expectedExistingSegments - .stream() - .collect(Collectors.toMap(DataSegment::getDataSource, segment -> 1L, Long::sum)); - final Map> expectedDataSources - = new HashMap<>(); + final Map expectedDataSourceSizes = + expectedExistingSegments.stream() + .collect(Collectors.toMap(DataSegment::getDataSource, DataSegment::getSize, Long::sum)); + final Map expectedDataSourceCounts = + expectedExistingSegments.stream() + .collect(Collectors.toMap(DataSegment::getDataSource, segment -> 1L, Long::sum)); + final Set expectedDataSourceNames = expectedExistingSegments.stream() + .map(DataSegment::getDataSource) + .collect(Collectors.toSet()); + final Map> expectedTimelines = new HashMap<>(); for (DataSegment segment : expectedExistingSegments) { final VersionedIntervalTimeline expectedTimeline = - expectedDataSources.computeIfAbsent( + expectedTimelines.computeIfAbsent( segment.getDataSource(), k -> new VersionedIntervalTimeline<>(Ordering.natural()) ); @@ -445,11 +447,12 @@ private void assertResult(List expectedExistingSegments) throws Seg ); } + Assert.assertEquals(expectedDataSourceNames, segmentManager.getDataSourceNames()); Assert.assertEquals(expectedDataSourceCounts, segmentManager.getDataSourceCounts()); Assert.assertEquals(expectedDataSourceSizes, segmentManager.getDataSourceSizes()); final Map dataSources = segmentManager.getDataSources(); - Assert.assertEquals(expectedDataSources.size(), dataSources.size()); + Assert.assertEquals(expectedTimelines.size(), dataSources.size()); dataSources.forEach( (sourceName, dataSourceState) -> { @@ -459,7 +462,7 @@ private void assertResult(List expectedExistingSegments) throws Seg dataSourceState.getTotalSegmentSize() ); Assert.assertEquals( - expectedDataSources.get(sourceName).getAllTimelineEntries(), + expectedTimelines.get(sourceName).getAllTimelineEntries(), dataSourceState.getTimeline().getAllTimelineEntries() ); } diff --git a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java index cc3a406cfe78..c11bb8179163 100644 --- a/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java +++ b/server/src/test/java/org/apache/druid/server/TestClusterQuerySegmentWalker.java @@ -45,8 +45,10 @@ import org.apache.druid.query.spec.SpecificSegmentSpec; import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; +import org.apache.druid.segment.SegmentReference; import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.segment.join.Joinables; +import org.apache.druid.segment.join.filter.rewrite.JoinFilterRewriteConfig; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; @@ -138,17 +140,19 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final && !toolChest.canPerformSubquery(((QueryDataSource) analysis.getDataSource()).getQuery())) { throw new ISE("Cannot handle subquery: %s", analysis.getDataSource()); } + final JoinFilterRewriteConfig joinFilterRewriteConfig = new JoinFilterRewriteConfig( + QueryContexts.getEnableJoinFilterPushDown(query), + QueryContexts.getEnableJoinFilterRewrite(query), + QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), + QueryContexts.getJoinFilterRewriteMaxSize(query) + ); - final Function segmentMapFn = Joinables.createSegmentMapFn( + final Function segmentMapFn = Joinables.createSegmentMapFn( analysis.getPreJoinableClauses(), joinableFactory, new AtomicLong(), - QueryContexts.getEnableJoinFilterPushDown(query), - QueryContexts.getEnableJoinFilterRewrite(query), - QueryContexts.getEnableJoinFilterRewriteValueColumnFilters(query), - QueryContexts.getJoinFilterRewriteMaxSize(query), - query.getFilter() == null ? null : query.getFilter().toFilter(), - query.getVirtualColumns() + joinFilterRewriteConfig, + query ); final QueryRunner baseRunner = new FinalizeResultsQueryRunner<>( @@ -197,7 +201,7 @@ private QueryRunner makeTableRunner( final QueryToolChest> toolChest, final QueryRunnerFactory> factory, final Iterable segments, - final Function segmentMapFn + final Function segmentMapFn ) { final List segmentsList = Lists.newArrayList(segments); @@ -217,7 +221,7 @@ private QueryRunner makeTableRunner( .transform( segment -> new SpecificSegmentQueryRunner<>( - factory.createRunner(segmentMapFn.apply(segment.getSegment())), + factory.createRunner(segmentMapFn.apply(ReferenceCountingSegment.wrapRootGenerationSegment(segment.getSegment()))), new SpecificSegmentSpec(segment.getDescriptor()) ) ) diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index f5455fbbfb29..6d8ef0a8cc4e 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.guice.ServerTypeConfig; +import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; @@ -32,6 +34,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.loading.CacheTestSegmentLoader; import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -39,12 +42,16 @@ import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.rules.TemporaryFolder; import java.io.File; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -67,6 +74,7 @@ public class SegmentLoadDropHandlerTest private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); private SegmentLoadDropHandler segmentLoadDropHandler; + private DataSegmentAnnouncer announcer; private File infoDir; private AtomicInteger announceCount; @@ -74,22 +82,36 @@ public class SegmentLoadDropHandlerTest private CacheTestSegmentLoader segmentLoader; private SegmentManager segmentManager; private List scheduledRunnable; + private SegmentLoaderConfig segmentLoaderConfig; + private SegmentLoaderConfig segmentLoaderConfigNoLocations; + private ScheduledExecutorFactory scheduledExecutorFactory; + private List locations; + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); @Before public void setUp() { try { - infoDir = new File(File.createTempFile("blah", "blah2").getParent(), "ZkCoordinatorTest"); - infoDir.mkdirs(); - for (File file : infoDir.listFiles()) { - file.delete(); - } + infoDir = temporaryFolder.newFolder(); log.info("Creating tmp test files in [%s]", infoDir); } catch (IOException e) { throw new RuntimeException(e); } + locations = Collections.singletonList( + new StorageLocationConfig( + infoDir, + 100L, + 100d + ) + ); + scheduledRunnable = new ArrayList<>(); segmentLoader = new CacheTestSegmentLoader(); @@ -132,57 +154,91 @@ public void unannounceSegments(Iterable segments) } }; - segmentLoadDropHandler = new SegmentLoadDropHandler( - jsonMapper, - new SegmentLoaderConfig() - { - @Override - public File getInfoDir() - { - return infoDir; - } - @Override - public int getNumLoadingThreads() - { - return 5; - } + segmentLoaderConfig = new SegmentLoaderConfig() + { + @Override + public File getInfoDir() + { + return infoDir; + } - @Override - public int getAnnounceIntervalMillis() - { - return 50; - } + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 50; + } + + @Override + public List getLocations() + { + return locations; + } + + @Override + public int getDropSegmentDelayMillis() + { + return 0; + } + }; + + segmentLoaderConfigNoLocations = new SegmentLoaderConfig() + { + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 50; + } + + @Override + public int getDropSegmentDelayMillis() + { + return 0; + } + }; + + scheduledExecutorFactory = new ScheduledExecutorFactory() + { + @Override + public ScheduledExecutorService create(int corePoolSize, String nameFormat) + { + /* + Override normal behavoir by adding the runnable to a list so that you can make sure + all the shceduled runnables are executed by explicitly calling run() on each item in the list + */ + return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat)) + { @Override - public int getDropSegmentDelayMillis() + public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - return 0; + scheduledRunnable.add(command); + return null; } - }, + }; + } + }; + + segmentLoadDropHandler = new SegmentLoadDropHandler( + jsonMapper, + segmentLoaderConfig, announcer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), segmentManager, - new ScheduledExecutorFactory() - { - @Override - public ScheduledExecutorService create(int corePoolSize, String nameFormat) - { - /* - Override normal behavoir by adding the runnable to a list so that you can make sure - all the shceduled runnables are executed by explicitly calling run() on each item in the list - */ - return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat)) - { - @Override - public ScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) - { - scheduledRunnable.add(command); - return null; - } - }; - } - }.create(5, "SegmentLoadDropHandlerTest-[%d]") + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + new ServerTypeConfig(ServerType.HISTORICAL) ); } @@ -220,6 +276,40 @@ Because another addSegment() call is executed, which removes the segment from se segmentLoadDropHandler.stop(); } + @Test + public void testSegmentLoading1BrokerWithNoLocations() throws Exception + { + SegmentLoadDropHandler segmentLoadDropHandlerBrokerWithNoLocations = new SegmentLoadDropHandler( + jsonMapper, + segmentLoaderConfigNoLocations, + announcer, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + segmentManager, + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-brokerNoLocations-[%d]"), + new ServerTypeConfig(ServerType.BROKER) + ); + + segmentLoadDropHandlerBrokerWithNoLocations.start(); + segmentLoadDropHandler.stop(); + } + + @Test + public void testSegmentLoading1HistoricalWithNoLocations() + { + expectedException.expect(IAE.class); + expectedException.expectMessage("Segment cache locations must be set on historicals."); + + new SegmentLoadDropHandler( + jsonMapper, + segmentLoaderConfigNoLocations, + announcer, + EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), + segmentManager, + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + new ServerTypeConfig(ServerType.HISTORICAL) + ); + } + /** * Steps: * 1. addSegment() succesfully loads the segment and annouces it @@ -382,13 +472,20 @@ public int getNumLoadingThreads() return 5; } + @Override + public List getLocations() + { + return locations; + } + @Override public int getAnnounceIntervalMillis() { return 50; } }, - announcer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), segmentManager + announcer, EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), segmentManager, + new ServerTypeConfig(ServerType.HISTORICAL) ); Set segments = new HashSet<>(); 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 356302cee228..9ca113e2515d 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 @@ -57,7 +57,6 @@ import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.search.SearchQuery; import org.apache.druid.query.search.SearchResultValue; -import org.apache.druid.segment.AbstractSegment; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.ReferenceCountingSegment; @@ -600,7 +599,7 @@ public TypeReference getResultTypeReference() } } - private static class SegmentForTesting extends AbstractSegment + private static class SegmentForTesting implements Segment { private final String version; private final Interval interval; diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java index 9bdd84ed70fb..c30a37cb01ad 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java @@ -23,10 +23,13 @@ import com.google.common.collect.ImmutableMap; import org.apache.curator.utils.ZKPaths; import org.apache.druid.curator.CuratorTestBase; +import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.IndexIO; 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.ServerTestHelper; import org.apache.druid.server.initialization.ZkPathsConfig; @@ -37,9 +40,15 @@ import org.easymock.EasyMock; import org.junit.After; import org.junit.Before; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import java.io.File; +import java.io.IOException; import java.util.Arrays; +import java.util.Collections; +import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ScheduledExecutorService; @@ -47,6 +56,8 @@ */ public class ZkCoordinatorTest extends CuratorTestBase { + private static final Logger log = new Logger(ZkCoordinatorTest.class); + private final ObjectMapper jsonMapper = ServerTestHelper.MAPPER; private final DruidServerMetadata me = new DruidServerMetadata( "dummyServer", @@ -67,9 +78,31 @@ public String getBase() }; private ZkCoordinator zkCoordinator; + private File infoDir; + private List locations; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Before public void setUp() throws Exception { + try { + infoDir = temporaryFolder.newFolder(); + log.info("Creating tmp test files in [%s]", infoDir); + } + catch (IOException e) { + throw new RuntimeException(e); + } + + locations = Collections.singletonList( + new StorageLocationConfig( + infoDir, + 100L, + 100d + ) + ); + setupServerAndCurator(); curator.start(); curator.blockUntilConnected(); @@ -102,11 +135,42 @@ public void testLoadDrop() throws Exception SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( ServerTestHelper.MAPPER, - new SegmentLoaderConfig(), + new SegmentLoaderConfig() { + @Override + public File getInfoDir() + { + return infoDir; + } + + @Override + public int getNumLoadingThreads() + { + return 5; + } + + @Override + public int getAnnounceIntervalMillis() + { + return 50; + } + + @Override + public List getLocations() + { + return locations; + } + + @Override + public int getDropSegmentDelayMillis() + { + return 0; + } + }, EasyMock.createNiceMock(DataSegmentAnnouncer.class), EasyMock.createNiceMock(DataSegmentServerAnnouncer.class), EasyMock.createNiceMock(SegmentManager.class), - EasyMock.createNiceMock(ScheduledExecutorService.class) + EasyMock.createNiceMock(ScheduledExecutorService.class), + new ServerTypeConfig(ServerType.HISTORICAL) ) { @Override diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java index 084a119ebe76..f37c92cb1056 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsTest.java @@ -43,6 +43,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; @@ -66,9 +67,11 @@ public class BalanceSegmentsTest private DataSegment segment2; private DataSegment segment3; private DataSegment segment4; + private DataSegment segment5; private List segments; private ListeningExecutorService balancerStrategyExecutor; private BalancerStrategy balancerStrategy; + private Set broadcastDatasources; @Before public void setUp() @@ -82,6 +85,7 @@ public void setUp() segment2 = EasyMock.createMock(DataSegment.class); segment3 = EasyMock.createMock(DataSegment.class); segment4 = EasyMock.createMock(DataSegment.class); + segment5 = EasyMock.createMock(DataSegment.class); DateTime start1 = DateTimes.of("2012-01-01"); DateTime start2 = DateTimes.of("2012-02-01"); @@ -130,12 +134,24 @@ public void setUp() 0, 8L ); + segment5 = new DataSegment( + "datasourceBroadcast", + new Interval(start2, start2.plusHours(1)), + version.toString(), + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + NoneShardSpec.instance(), + 0, + 8L + ); segments = new ArrayList<>(); segments.add(segment1); segments.add(segment2); segments.add(segment3); segments.add(segment4); + segments.add(segment5); peon1 = new LoadQueuePeonTester(); peon2 = new LoadQueuePeonTester(); @@ -147,6 +163,8 @@ public void setUp() balancerStrategyExecutor = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(balancerStrategyExecutor); + + broadcastDatasources = Collections.singleton("datasourceBroadcast"); } @After @@ -187,10 +205,11 @@ public void testMoveToEmptyServerBalancer() ImmutableList.of(peon1, peon2) ) .withBalancerStrategy(predefinedPickOrderStrategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); - Assert.assertEquals(2, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); + Assert.assertEquals(3, params.getCoordinatorStats().getTieredStat("movedCount", "normal")); } /** @@ -213,10 +232,10 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove() mockCoordinator(coordinator); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, false)))) + EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, false)), broadcastDatasources)) .andReturn(new BalancerSegmentHolder(druidServer2, segment3)) .andReturn(new BalancerSegmentHolder(druidServer2, segment4)); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .andReturn(new BalancerSegmentHolder(druidServer1, segment2)); @@ -237,6 +256,7 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMove() .build() // ceil(3 * 0.6) = 2 segments from decommissioning servers ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -280,7 +300,7 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissi mockCoordinator(coordinator); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .andReturn(new BalancerSegmentHolder(druidServer1, segment2)) .andReturn(new BalancerSegmentHolder(druidServer2, segment3)) @@ -303,6 +323,7 @@ public void testMoveDecommissioningMaxPercentOfMaxSegmentsToMoveWithNoDecommissi .build() ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -328,7 +349,7 @@ public void testMoveToDecommissioningServer() mockCoordinator(coordinator); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .anyTimes(); EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())).andAnswer(() -> { @@ -343,6 +364,7 @@ public void testMoveToDecommissioningServer() ImmutableList.of(false, true) ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -362,7 +384,7 @@ public void testMoveFromDecommissioningServer() ServerHolder holder2 = new ServerHolder(druidServer2, peon2, false); BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)) .once(); EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) @@ -377,6 +399,7 @@ public void testMoveFromDecommissioningServer() ) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(1).build()) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); params = new BalanceSegmentsTester(coordinator).run(params); @@ -412,6 +435,7 @@ public void testMoveMaxLoadQueueServerBalancer() ImmutableList.of(peon1, peon2) ) .withBalancerStrategy(predefinedPickOrderStrategy) + .withBroadcastDatasources(broadcastDatasources) .withDynamicConfigs( CoordinatorDynamicConfig .builder() @@ -451,6 +475,7 @@ public void testMoveSameSegmentTwice() ImmutableList.of(peon1, peon2) ) .withBalancerStrategy(predefinedPickOrderStrategy) + .withBroadcastDatasources(broadcastDatasources) .withDynamicConfigs( CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( 2 @@ -542,6 +567,7 @@ private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( ) .withUsedSegmentsInTest(segments) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) + .withBroadcastDatasources(broadcastDatasources) .withBalancerStrategy(balancerStrategy); } @@ -611,7 +637,7 @@ public ServerHolder findNewSegmentHomeReplicator(DataSegment proposalSegment, Li } @Override - public BalancerSegmentHolder pickSegmentToMove(List serverHolders) + public BalancerSegmentHolder pickSegmentToMove(List serverHolders, Set broadcastDatasources) { return pickOrder.get(pickCounter.getAndIncrement() % pickOrder.size()); } @@ -635,9 +661,9 @@ private DruidCoordinatorRuntimeParams setupParamsForDecommissioningMaxPercentOfM // either decommissioning servers list or acitve ones (ie servers list is [2] or [1, 3]) BalancerStrategy strategy = EasyMock.createMock(BalancerStrategy.class); - EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, true)))) + EasyMock.expect(strategy.pickSegmentToMove(ImmutableList.of(new ServerHolder(druidServer2, peon2, true)), broadcastDatasources)) .andReturn(new BalancerSegmentHolder(druidServer2, segment2)); - EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject())) + EasyMock.expect(strategy.pickSegmentToMove(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new BalancerSegmentHolder(druidServer1, segment1)); EasyMock.expect(strategy.findNewSegmentHomeBalancer(EasyMock.anyObject(), EasyMock.anyObject())) .andReturn(new ServerHolder(druidServer3, peon3)) @@ -656,6 +682,7 @@ private DruidCoordinatorRuntimeParams setupParamsForDecommissioningMaxPercentOfM .build() ) .withBalancerStrategy(strategy) + .withBroadcastDatasources(broadcastDatasources) .build(); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java index 772b7aec1401..5fb100073eae 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java @@ -35,6 +35,7 @@ public static DruidClusterBuilder newBuilder() private @Nullable Set realtimes = null; private final Map> historicals = new HashMap<>(); + private @Nullable Set brokers = null; private DruidClusterBuilder() { @@ -46,6 +47,12 @@ public DruidClusterBuilder withRealtimes(ServerHolder... realtimes) return this; } + public DruidClusterBuilder withBrokers(ServerHolder... brokers) + { + this.brokers = new HashSet<>(Arrays.asList(brokers)); + return this; + } + public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals) { if (this.historicals.putIfAbsent(tierName, Arrays.asList(historicals)) != null) { @@ -56,6 +63,6 @@ public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals) public DruidCluster build() { - return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals); + return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals, brokers); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index e48bf9430211..0077fc1123a4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -28,8 +28,6 @@ import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; -import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; -import org.apache.curator.utils.ZKPaths; import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidServer; @@ -52,6 +50,7 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; import org.apache.druid.server.coordinator.rules.IntervalLoadRule; import org.apache.druid.server.coordinator.rules.Rule; @@ -104,10 +103,12 @@ public class DruidCoordinatorTest extends CuratorTestBase private ObjectMapper objectMapper; private DruidNode druidNode; private LatchableServiceEmitter serviceEmitter = new LatchableServiceEmitter(); + private boolean serverAddedCountExpected = true; @Before public void setUp() throws Exception { + serverAddedCountExpected = true; druidServer = EasyMock.createMock(DruidServer.class); serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); segmentsMetadataManager = EasyMock.createNiceMock(SegmentsMetadataManager.class); @@ -375,37 +376,22 @@ public void testCoordinatorRun() throws Exception // This coordinator should be leader by now Assert.assertTrue(coordinator.isLeader()); Assert.assertEquals(druidNode.getHostAndPort(), coordinator.getCurrentLeader()); - - final CountDownLatch assignSegmentLatch = new CountDownLatch(1); - pathChildrenCache.getListenable().addListener( - new PathChildrenCacheListener() - { - @Override - public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent event) - { - if (CuratorUtils.isChildAdded(event)) { - if (assignSegmentLatch.getCount() > 0) { - //Coordinator should try to assign segment to druidServer historical - //Simulate historical loading segment - druidServer.addDataSegment(dataSegment); - assignSegmentLatch.countDown(); - } else { - Assert.fail("The same segment is assigned to the same server multiple times"); - } - } - } - } - ); pathChildrenCache.start(); + final CountDownLatch assignSegmentLatch = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch( + 1, + pathChildrenCache, + ImmutableMap.of("2010-01-01T00:00:00.000Z_2010-01-02T00:00:00.000Z", dataSegment), + druidServer + ); assignSegmentLatch.await(); + Assert.assertTrue(serverAddedCountExpected); final CountDownLatch coordinatorRunLatch = new CountDownLatch(2); serviceEmitter.latch = coordinatorRunLatch; coordinatorRunLatch.await(); Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); - curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getId().toString())); Object2IntMap numsUnavailableUsedSegmentsPerDataSource = coordinator.computeNumsUnavailableUsedSegmentsPerDataSource(); @@ -496,39 +482,171 @@ public void testCoordinatorTieredRun() throws Exception coordinator.start(); leaderAnnouncerLatch.await(); // Wait for this coordinator to become leader - final CountDownLatch assignSegmentLatchHot = new CountDownLatch(2); - pathChildrenCache.getListenable().addListener( - (client, event) -> { - if (CuratorUtils.isChildAdded(event)) { - DataSegment segment = findSegmentRelatedToCuratorEvent(dataSegments, event); - if (segment != null) { - hotServer.addDataSegment(segment); - curator.delete().guaranteed().forPath(event.getData().getPath()); - } + final CountDownLatch assignSegmentLatchHot = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(2, pathChildrenCache, dataSegments, hotServer); + final CountDownLatch assignSegmentLatchCold = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(1, pathChildrenCacheCold, dataSegments, coldServer); + assignSegmentLatchHot.await(); + assignSegmentLatchCold.await(); + Assert.assertTrue(serverAddedCountExpected); - assignSegmentLatchHot.countDown(); - } - } + final CountDownLatch coordinatorRunLatch = new CountDownLatch(2); + serviceEmitter.latch = coordinatorRunLatch; + coordinatorRunLatch.await(); + + Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); + + Map> underReplicationCountsPerDataSourcePerTier = + coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); + Assert.assertEquals(2, underReplicationCountsPerDataSourcePerTier.size()); + Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(hotTierName).getLong(dataSource)); + Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(coldTierName).getLong(dataSource)); + + coordinator.stop(); + leaderUnannouncerLatch.await(); + + EasyMock.verify(serverInventoryView); + EasyMock.verify(segmentsMetadataManager); + EasyMock.verify(metadataRuleManager); + } + + @Test(timeout = 60_000L) + public void testComputeUnderReplicationCountsPerDataSourcePerTierForSegmentsWithBroadcastRule() throws Exception + { + final String dataSource = "dataSource"; + final String hotTierName = "hot"; + final String coldTierName = "cold"; + final String tierName1 = "tier1"; + final String tierName2 = "tier2"; + final Rule broadcastDistributionRule = new ForeverBroadcastDistributionRule(); + final String loadPathCold = "/druid/loadqueue/cold:1234"; + final String loadPathBroker1 = "/druid/loadqueue/broker1:1234"; + final String loadPathBroker2 = "/druid/loadqueue/broker2:1234"; + final String loadPathPeon = "/druid/loadqueue/peon:1234"; + final DruidServer hotServer = new DruidServer("hot", "hot", null, 5L, ServerType.HISTORICAL, hotTierName, 0); + final DruidServer coldServer = new DruidServer("cold", "cold", null, 5L, ServerType.HISTORICAL, coldTierName, 0); + final DruidServer brokerServer1 = new DruidServer("broker1", "broker1", null, 5L, ServerType.BROKER, tierName1, 0); + final DruidServer brokerServer2 = new DruidServer("broker2", "broker2", null, 5L, ServerType.BROKER, tierName2, 0); + final DruidServer peonServer = new DruidServer("peon", "peon", null, 5L, ServerType.INDEXER_EXECUTOR, tierName2, 0); + + final Map dataSegments = ImmutableMap.of( + "2018-01-02T00:00:00.000Z_2018-01-03T00:00:00.000Z", + new DataSegment(dataSource, Intervals.of("2018-01-02/P1D"), "v1", null, null, null, null, 0x9, 0), + "2018-01-03T00:00:00.000Z_2018-01-04T00:00:00.000Z", + new DataSegment(dataSource, Intervals.of("2018-01-03/P1D"), "v1", null, null, null, null, 0x9, 0), + "2017-01-01T00:00:00.000Z_2017-01-02T00:00:00.000Z", + new DataSegment(dataSource, Intervals.of("2017-01-01/P1D"), "v1", null, null, null, null, 0x9, 0) ); - final CountDownLatch assignSegmentLatchCold = new CountDownLatch(1); - pathChildrenCacheCold.getListenable().addListener( - (CuratorFramework client, PathChildrenCacheEvent event) -> { - if (CuratorUtils.isChildAdded(event)) { - DataSegment segment = findSegmentRelatedToCuratorEvent(dataSegments, event); + final LoadQueuePeon loadQueuePeonCold = new CuratorLoadQueuePeon( + curator, + loadPathCold, + objectMapper, + Execs.scheduledSingleThreaded("coordinator_test_load_queue_peon_cold_scheduled-%d"), + Execs.singleThreaded("coordinator_test_load_queue_peon_cold-%d"), + druidCoordinatorConfig + ); - if (segment != null) { - coldServer.addDataSegment(segment); - curator.delete().guaranteed().forPath(event.getData().getPath()); - } + final LoadQueuePeon loadQueuePeonBroker1 = new CuratorLoadQueuePeon( + curator, + loadPathBroker1, + objectMapper, + Execs.scheduledSingleThreaded("coordinator_test_load_queue_peon_broker1_scheduled-%d"), + Execs.singleThreaded("coordinator_test_load_queue_peon_broker1-%d"), + druidCoordinatorConfig + ); - assignSegmentLatchCold.countDown(); - } - } + final LoadQueuePeon loadQueuePeonBroker2 = new CuratorLoadQueuePeon( + curator, + loadPathBroker2, + objectMapper, + Execs.scheduledSingleThreaded("coordinator_test_load_queue_peon_broker2_scheduled-%d"), + Execs.singleThreaded("coordinator_test_load_queue_peon_broker2-%d"), + druidCoordinatorConfig + ); + + final LoadQueuePeon loadQueuePeonPoenServer = new CuratorLoadQueuePeon( + curator, + loadPathPeon, + objectMapper, + Execs.scheduledSingleThreaded("coordinator_test_load_queue_peon_peon_scheduled-%d"), + Execs.singleThreaded("coordinator_test_load_queue_peon_peon-%d"), + druidCoordinatorConfig + ); + final PathChildrenCache pathChildrenCacheCold = new PathChildrenCache( + curator, + loadPathCold, + true, + true, + Execs.singleThreaded("coordinator_test_path_children_cache_cold-%d") + ); + final PathChildrenCache pathChildrenCacheBroker1 = new PathChildrenCache( + curator, + loadPathBroker1, + true, + true, + Execs.singleThreaded("coordinator_test_path_children_cache_broker1-%d") + ); + final PathChildrenCache pathChildrenCacheBroker2 = new PathChildrenCache( + curator, + loadPathBroker2, + true, + true, + Execs.singleThreaded("coordinator_test_path_children_cache_broker2-%d") + ); + final PathChildrenCache pathChildrenCachePeon = new PathChildrenCache( + curator, + loadPathPeon, + true, + true, + Execs.singleThreaded("coordinator_test_path_children_cache_peon-%d") ); + loadManagementPeons.putAll(ImmutableMap.of("hot", loadQueuePeon, + "cold", loadQueuePeonCold, + "broker1", loadQueuePeonBroker1, + "broker2", loadQueuePeonBroker2, + "peon", loadQueuePeonPoenServer)); + + loadQueuePeonCold.start(); + loadQueuePeonBroker1.start(); + loadQueuePeonBroker2.start(); + loadQueuePeonPoenServer.start(); + pathChildrenCache.start(); + pathChildrenCacheCold.start(); + pathChildrenCacheBroker1.start(); + pathChildrenCacheBroker2.start(); + pathChildrenCachePeon.start(); + + DruidDataSource[] druidDataSources = {new DruidDataSource(dataSource, Collections.emptyMap())}; + dataSegments.values().forEach(druidDataSources[0]::addSegment); + + setupSegmentsMetadataMock(druidDataSources[0]); + + EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) + .andReturn(ImmutableList.of(broadcastDistributionRule)).atLeastOnce(); + EasyMock.expect(metadataRuleManager.getAllRules()) + .andReturn(ImmutableMap.of(dataSource, ImmutableList.of(broadcastDistributionRule))).atLeastOnce(); + + EasyMock.expect(serverInventoryView.getInventory()) + .andReturn(ImmutableList.of(hotServer, coldServer, brokerServer1, brokerServer2, peonServer)) + .atLeastOnce(); + EasyMock.expect(serverInventoryView.isStarted()).andReturn(true).anyTimes(); + + EasyMock.replay(metadataRuleManager, serverInventoryView); + + coordinator.start(); + leaderAnnouncerLatch.await(); // Wait for this coordinator to become leader + + final CountDownLatch assignSegmentLatchHot = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(3, pathChildrenCache, dataSegments, hotServer); + final CountDownLatch assignSegmentLatchCold = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(3, pathChildrenCacheCold, dataSegments, coldServer); + final CountDownLatch assignSegmentLatchBroker1 = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(3, pathChildrenCacheBroker1, dataSegments, brokerServer1); + final CountDownLatch assignSegmentLatchBroker2 = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(3, pathChildrenCacheBroker2, dataSegments, brokerServer2); + final CountDownLatch assignSegmentLatchPeon = createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(3, pathChildrenCachePeon, dataSegments, peonServer); assignSegmentLatchHot.await(); assignSegmentLatchCold.await(); + assignSegmentLatchBroker1.await(); + assignSegmentLatchBroker2.await(); + assignSegmentLatchPeon.await(); + Assert.assertTrue(serverAddedCountExpected); final CountDownLatch coordinatorRunLatch = new CountDownLatch(2); serviceEmitter.latch = coordinatorRunLatch; @@ -538,9 +656,11 @@ public void testCoordinatorTieredRun() throws Exception Map> underReplicationCountsPerDataSourcePerTier = coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); - Assert.assertEquals(2, underReplicationCountsPerDataSourcePerTier.size()); + Assert.assertEquals(4, underReplicationCountsPerDataSourcePerTier.size()); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(hotTierName).getLong(dataSource)); Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(coldTierName).getLong(dataSource)); + Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(tierName1).getLong(dataSource)); + Assert.assertEquals(0L, underReplicationCountsPerDataSourcePerTier.get(tierName2).getLong(dataSource)); coordinator.stop(); leaderUnannouncerLatch.await(); @@ -550,6 +670,32 @@ public void testCoordinatorTieredRun() throws Exception EasyMock.verify(metadataRuleManager); } + private CountDownLatch createCountDownLatchAndSetPathChildrenCacheListenerWithLatch(int latchCount, + PathChildrenCache pathChildrenCache, + Map segments, + DruidServer server) + { + final CountDownLatch countDownLatch = new CountDownLatch(latchCount); + pathChildrenCache.getListenable().addListener( + (CuratorFramework client, PathChildrenCacheEvent event) -> { + if (CuratorUtils.isChildAdded(event)) { + if (countDownLatch.getCount() > 0) { + DataSegment segment = findSegmentRelatedToCuratorEvent(segments, event); + if (segment != null) { + server.addDataSegment(segment); + curator.delete().guaranteed().forPath(event.getData().getPath()); + } + countDownLatch.countDown(); + } else { + // The segment is assigned to the server more times than expected + serverAddedCountExpected = false; + } + } + } + ); + return countDownLatch; + } + private void setupSegmentsMetadataMock(DruidDataSource dataSource) { EasyMock.expect(segmentsMetadataManager.isPollingDatabasePeriodically()).andReturn(true).anyTimes(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java index 73e829ce0e0c..8aef2f2e10fb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java @@ -23,6 +23,7 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -136,6 +137,7 @@ public void setUp() @Test public void getRandomBalancerSegmentHolderTest() { + EasyMock.expect(druidServer1.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); @@ -143,6 +145,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); + EasyMock.expect(druidServer2.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer2.getName()).andReturn("2").atLeastOnce(); EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(30L).atLeastOnce(); @@ -151,6 +154,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); + EasyMock.expect(druidServer3.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer3.getName()).andReturn("3").atLeastOnce(); EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer3.getCurrSize()).andReturn(30L).atLeastOnce(); @@ -159,6 +163,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer3.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer3); + EasyMock.expect(druidServer4.getType()).andReturn(ServerType.HISTORICAL).atLeastOnce(); EasyMock.expect(druidServer4.getName()).andReturn("4").atLeastOnce(); EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer4.getCurrSize()).andReturn(30L).atLeastOnce(); @@ -186,7 +191,7 @@ public void getRandomBalancerSegmentHolderTest() Map segmentCountMap = new HashMap<>(); for (int i = 0; i < 5000; i++) { - segmentCountMap.put(ReservoirSegmentSampler.getRandomBalancerSegmentHolder(holderList).getSegment(), 1); + segmentCountMap.put(ReservoirSegmentSampler.getRandomBalancerSegmentHolder(holderList, Collections.emptySet()).getSegment(), 1); } for (DataSegment segment : segments) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java index fcebbdee8209..cb1ee0425b61 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ServerHolderTest.java @@ -39,7 +39,7 @@ public class ServerHolderTest { private static final List SEGMENTS = ImmutableList.of( new DataSegment( - "test", + "src1", Intervals.of("2015-04-12/2015-04-13"), "1", ImmutableMap.of("containerName", "container1", "blobPath", "blobPath1"), @@ -50,7 +50,7 @@ public class ServerHolderTest 1 ), new DataSegment( - "test", + "src2", Intervals.of("2015-04-12/2015-04-13"), "1", ImmutableMap.of("containerName", "container2", "blobPath", "blobPath2"), @@ -177,4 +177,22 @@ public void testEquals() Assert.assertNotEquals(h1, h4); Assert.assertNotEquals(h1, h5); } + + @Test + public void testIsServingSegment() + { + final ServerHolder h1 = new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of("src1", DATA_SOURCES.get("src1")), + 1 + ), + new LoadQueuePeonTester() + ); + Assert.assertTrue(h1.isServingSegment(SEGMENTS.get(0))); + Assert.assertFalse(h1.isServingSegment(SEGMENTS.get(1))); + Assert.assertTrue(h1.isServingSegment(SEGMENTS.get(0).getId())); + Assert.assertFalse(h1.isServingSegment(SEGMENTS.get(1).getId())); + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java index 73b95ee960ce..301b3bd8f9f9 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkAsUnusedOvershadowedSegmentsTest.java @@ -21,6 +21,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.DateTimes; @@ -39,9 +41,11 @@ import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Test; +import org.junit.runner.RunWith; import java.util.List; +@RunWith(JUnitParamsRunner.class) public class MarkAsUnusedOvershadowedSegmentsTest { MarkAsUnusedOvershadowedSegments markAsUnusedOvershadowedSegments; @@ -69,8 +73,16 @@ public class MarkAsUnusedOvershadowedSegmentsTest .build(); @Test - public void testRun() + @Parameters( + { + "historical", + "broker" + } + ) + public void testRun(String serverTypeString) { + ServerType serverType = ServerType.fromString(serverTypeString); + markAsUnusedOvershadowedSegments = new MarkAsUnusedOvershadowedSegments(coordinator); usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); @@ -95,7 +107,7 @@ public void testRun() .andReturn("") .anyTimes(); EasyMock.expect(druidServer.getType()) - .andReturn(ServerType.HISTORICAL) + .andReturn(serverType) .anyTimes(); EasyMock.expect(druidServer.getDataSources()) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java new file mode 100644 index 000000000000..f74762c7ad9d --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java @@ -0,0 +1,364 @@ +/* + * 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.coordinator.duty; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.metadata.MetadataRuleManager; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; +import org.apache.druid.server.coordinator.CoordinatorStats; +import org.apache.druid.server.coordinator.DruidClusterBuilder; +import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; +import org.apache.druid.server.coordinator.LoadQueuePeonTester; +import org.apache.druid.server.coordinator.ServerHolder; +import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; +import org.apache.druid.server.coordinator.rules.ForeverLoadRule; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.partition.NoneShardSpec; +import org.easymock.EasyMock; +import org.joda.time.DateTime; +import org.joda.time.Interval; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Set; + +public class UnloadUnusedSegmentsTest +{ + private DruidCoordinator coordinator; + private ImmutableDruidServer historicalServer; + private ImmutableDruidServer historicalServerTier2; + private ImmutableDruidServer brokerServer; + private ImmutableDruidServer indexerServer; + private LoadQueuePeonTester historicalPeon; + private LoadQueuePeonTester historicalTier2Peon; + private LoadQueuePeonTester brokerPeon; + private LoadQueuePeonTester indexerPeon; + private DataSegment segment1; + private DataSegment segment2; + private DataSegment broadcastSegment; + private DataSegment realtimeOnlySegment; + private List segments; + private List segmentsForRealtime; + private ImmutableDruidDataSource dataSource1; + private ImmutableDruidDataSource dataSource2; + private ImmutableDruidDataSource dataSource2ForRealtime; + private ImmutableDruidDataSource broadcastDatasource; + private List dataSources; + private List dataSourcesForRealtime; + private Set broadcastDatasourceNames; + private MetadataRuleManager databaseRuleManager; + + @Before + public void setUp() + { + coordinator = EasyMock.createMock(DruidCoordinator.class); + historicalServer = EasyMock.createMock(ImmutableDruidServer.class); + historicalServerTier2 = EasyMock.createMock(ImmutableDruidServer.class); + brokerServer = EasyMock.createMock(ImmutableDruidServer.class); + indexerServer = EasyMock.createMock(ImmutableDruidServer.class); + segment1 = EasyMock.createMock(DataSegment.class); + segment2 = EasyMock.createMock(DataSegment.class); + databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); + + DateTime start1 = DateTimes.of("2012-01-01"); + DateTime start2 = DateTimes.of("2012-02-01"); + DateTime version = DateTimes.of("2012-05-01"); + segment1 = new DataSegment( + "datasource1", + new Interval(start1, start1.plusHours(1)), + version.toString(), + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + NoneShardSpec.instance(), + 0, + 11L + ); + segment2 = new DataSegment( + "datasource2", + new Interval(start1, start1.plusHours(1)), + version.toString(), + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + NoneShardSpec.instance(), + 0, + 7L + ); + realtimeOnlySegment = new DataSegment( + "datasource2", + new Interval(start2, start2.plusHours(1)), + version.toString(), + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + NoneShardSpec.instance(), + 0, + 7L + ); + broadcastSegment = new DataSegment( + "broadcastDatasource", + new Interval(start1, start1.plusHours(1)), + version.toString(), + new HashMap<>(), + new ArrayList<>(), + new ArrayList<>(), + NoneShardSpec.instance(), + 0, + 7L + ); + + segments = new ArrayList<>(); + segments.add(segment1); + segments.add(segment2); + segments.add(broadcastSegment); + + segmentsForRealtime = new ArrayList<>(); + segmentsForRealtime.add(realtimeOnlySegment); + segmentsForRealtime.add(broadcastSegment); + + historicalPeon = new LoadQueuePeonTester(); + historicalTier2Peon = new LoadQueuePeonTester(); + brokerPeon = new LoadQueuePeonTester(); + indexerPeon = new LoadQueuePeonTester(); + + dataSource1 = new ImmutableDruidDataSource( + "datasource1", + Collections.emptyMap(), + Collections.singleton(segment1) + ); + dataSource2 = new ImmutableDruidDataSource( + "datasource2", + Collections.emptyMap(), + Collections.singleton(segment2) + ); + + broadcastDatasourceNames = Collections.singleton("broadcastDatasource"); + broadcastDatasource = new ImmutableDruidDataSource( + "broadcastDatasource", + Collections.emptyMap(), + Collections.singleton(broadcastSegment) + ); + + dataSources = ImmutableList.of(dataSource1, dataSource2, broadcastDatasource); + + // This simulates a task that is ingesting to an existing non-broadcast datasource, with unpublished segments, + // while also having a broadcast segment loaded. + dataSource2ForRealtime = new ImmutableDruidDataSource( + "datasource2", + Collections.emptyMap(), + Collections.singleton(realtimeOnlySegment) + ); + dataSourcesForRealtime = ImmutableList.of(dataSource2ForRealtime, broadcastDatasource); + } + + @After + public void tearDown() + { + EasyMock.verify(coordinator); + EasyMock.verify(historicalServer); + EasyMock.verify(historicalServerTier2); + EasyMock.verify(brokerServer); + EasyMock.verify(indexerServer); + EasyMock.verify(databaseRuleManager); + } + + @Test + public void test_unloadUnusedSegmentsFromAllServers() + { + mockDruidServer( + historicalServer, + ServerType.HISTORICAL, + "historical", + DruidServer.DEFAULT_TIER, + 30L, + 100L, + segments, + dataSources + ); + mockDruidServer( + historicalServerTier2, + ServerType.HISTORICAL, + "historicalTier2", + "tier2", + 30L, + 100L, + segments, + dataSources + ); + mockDruidServer( + brokerServer, + ServerType.BROKER, + "broker", + DruidServer.DEFAULT_TIER, + 30L, + 100L, + segments, + dataSources + ); + mockDruidServer( + indexerServer, + ServerType.INDEXER_EXECUTOR, + "indexer", + DruidServer.DEFAULT_TIER, + 30L, + 100L, + segmentsForRealtime, + dataSourcesForRealtime + ); + + // Mock stuff that the coordinator needs + mockCoordinator(coordinator); + + mockRuleManager(databaseRuleManager); + + // We keep datasource2 segments only, drop datasource1 and broadcastDatasource from all servers + // realtimeSegment is intentionally missing from the set, to match how a realtime tasks's unpublished segments + // will not appear in the coordinator's view of used segments. + Set usedSegments = ImmutableSet.of(segment2); + + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster( + DruidClusterBuilder + .newBuilder() + .addTier( + DruidServer.DEFAULT_TIER, + new ServerHolder(historicalServer, historicalPeon, false) + ) + .addTier( + "tier2", + new ServerHolder(historicalServerTier2, historicalTier2Peon, false) + ) + .withBrokers( + new ServerHolder(brokerServer, brokerPeon, false) + ) + .withRealtimes( + new ServerHolder(indexerServer, indexerPeon, false) + ) + .build() + ) + .withLoadManagementPeons( + ImmutableMap.of( + "historical", historicalPeon, + "historicalTier2", historicalTier2Peon, + "broker", brokerPeon, + "indexer", indexerPeon + ) + ) + .withUsedSegmentsInTest(usedSegments) + .withBroadcastDatasources(broadcastDatasourceNames) + .withDatabaseRuleManager(databaseRuleManager) + .build(); + + params = new UnloadUnusedSegments().run(params); + CoordinatorStats stats = params.getCoordinatorStats(); + + // We drop segment1 and broadcast1 from all servers, realtimeSegment is not dropped by the indexer + Assert.assertEquals(5, stats.getTieredStat("unneededCount", DruidServer.DEFAULT_TIER)); + Assert.assertEquals(2, stats.getTieredStat("unneededCount", "tier2")); + } + + private static void mockDruidServer( + ImmutableDruidServer druidServer, + ServerType serverType, + String name, + String tier, + long currentSize, + long maxSize, + List segments, + List dataSources + ) + { + EasyMock.expect(druidServer.getName()).andReturn(name).anyTimes(); + EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes(); + EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).anyTimes(); + EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer, segments); + EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes(); + EasyMock.expect(druidServer.getType()).andReturn(serverType).anyTimes(); + EasyMock.expect(druidServer.getDataSources()).andReturn(dataSources).anyTimes(); + if (!segments.isEmpty()) { + segments.forEach( + s -> EasyMock.expect(druidServer.getSegment(s.getId())).andReturn(s).anyTimes() + ); + } + EasyMock.expect(druidServer.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); + EasyMock.replay(druidServer); + } + + private static void mockCoordinator(DruidCoordinator coordinator) + { + coordinator.moveSegment( + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject(), + EasyMock.anyObject() + ); + EasyMock.expectLastCall().anyTimes(); + EasyMock.replay(coordinator); + } + + private static void mockRuleManager(MetadataRuleManager metadataRuleManager) + { + EasyMock.expect(metadataRuleManager.getRulesWithDefault("datasource1")).andReturn( + Collections.singletonList( + new ForeverLoadRule( + ImmutableMap.of( + DruidServer.DEFAULT_TIER, 1, + "tier2", 1 + ) + ) + )).anyTimes(); + + EasyMock.expect(metadataRuleManager.getRulesWithDefault("datasource2")).andReturn( + Collections.singletonList( + new ForeverLoadRule( + ImmutableMap.of( + DruidServer.DEFAULT_TIER, 1, + "tier2", 1 + ) + ) + )).anyTimes(); + + EasyMock.expect(metadataRuleManager.getRulesWithDefault("broadcastDatasource")).andReturn( + Collections.singletonList( + new ForeverBroadcastDistributionRule() + )).anyTimes(); + + EasyMock.replay(metadataRuleManager); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java index e3b51a51b02d..f2405060fea8 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleSerdeTest.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; @@ -44,15 +43,9 @@ public class BroadcastDistributionRuleSerdeTest public static List constructorFeeder() { return Lists.newArrayList( - new Object[]{new ForeverBroadcastDistributionRule(ImmutableList.of("large_source1", "large_source2"))}, - new Object[]{new ForeverBroadcastDistributionRule(ImmutableList.of())}, - new Object[]{new ForeverBroadcastDistributionRule(null)}, - new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"), ImmutableList.of("large_source"))}, - new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"), ImmutableList.of())}, - new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"), null)}, - new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null, ImmutableList.of("large_source"))}, - new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null, ImmutableList.of())}, - new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null, null)} + new Object[]{new ForeverBroadcastDistributionRule()}, + new Object[]{new IntervalBroadcastDistributionRule(Intervals.of("0/1000"))}, + new Object[]{new PeriodBroadcastDistributionRule(new Period(1000), null)} ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 70ec3ebec052..c2d4fd3d0004 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -19,7 +19,6 @@ package org.apache.druid.server.coordinator.rules; -import com.google.common.collect.ImmutableList; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -269,7 +268,7 @@ public void setUp() public void testBroadcastToSingleDataSource() { final ForeverBroadcastDistributionRule rule = - new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + new ForeverBroadcastDistributionRule(); CoordinatorStats stats = rule.run( null, @@ -285,7 +284,7 @@ public void testBroadcastToSingleDataSource() smallSegment ); - Assert.assertEquals(3L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); Assert.assertFalse(stats.hasPerTierStats()); Assert.assertTrue( @@ -295,10 +294,10 @@ public void testBroadcastToSingleDataSource() Assert.assertTrue( holdersOfLargeSegments2.stream() - .noneMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) ); - Assert.assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment)); + Assert.assertTrue(holderOfSmallSegment.isServingSegment(smallSegment)); } private static DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( @@ -331,7 +330,7 @@ private static DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( public void testBroadcastDecommissioning() { final ForeverBroadcastDistributionRule rule = - new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + new ForeverBroadcastDistributionRule(); CoordinatorStats stats = rule.run( null, @@ -356,7 +355,6 @@ public void testBroadcastDecommissioning() public void testBroadcastToMultipleDataSources() { final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule( - ImmutableList.of("large_source", "large_source2") ); CoordinatorStats stats = rule.run( @@ -392,7 +390,7 @@ public void testBroadcastToMultipleDataSources() @Test public void testBroadcastToAllServers() { - final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(null); + final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(); CoordinatorStats stats = rule.run( null, @@ -408,14 +406,14 @@ public void testBroadcastToAllServers() smallSegment ); - Assert.assertEquals(6L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); + Assert.assertEquals(5L, stats.getGlobalStat(LoadRule.ASSIGNED_COUNT)); Assert.assertFalse(stats.hasPerTierStats()); Assert.assertTrue( druidCluster .getAllServers() .stream() - .allMatch(holder -> holder.getPeon().getSegmentsToLoad().contains(smallSegment)) + .allMatch(holder -> holder.isLoadingSegment(smallSegment) || holder.isServingSegment(smallSegment)) ); } } diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index 10a7f97300c9..39e02ae86def 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -19,11 +19,14 @@ package org.apache.druid.server.http; +import com.google.common.base.Optional; 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.Sets; +import it.unimi.dsi.fastutil.objects.Object2LongMap; +import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidServer; @@ -39,6 +42,7 @@ import org.apache.druid.query.TableDataSource; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.rules.IntervalDropRule; import org.apache.druid.server.coordinator.rules.IntervalLoadRule; import org.apache.druid.server.coordinator.rules.Rule; @@ -176,7 +180,7 @@ public void testGetFullQueryableDataSources() EasyMock.replay(inventoryView, server, request); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null); Response response = dataSourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); Assert.assertEquals(200, response.getStatus()); @@ -250,7 +254,7 @@ public Access authorize(AuthenticationResult authenticationResult1, Resource res } }; - DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, authMapper); + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, authMapper, null); Response response = dataSourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); @@ -289,7 +293,7 @@ public void testGetSimpleQueryableDataSources() EasyMock.replay(inventoryView, server, request); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER); + new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER, null); Response response = dataSourcesResource.getQueryableDataSources(null, "simple", request); Assert.assertEquals(200, response.getStatus()); List> results = (List>) response.getEntity(); @@ -313,7 +317,7 @@ public void testFullGetTheDataSource() EasyMock.replay(inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, null); + new DataSourcesResource(inventoryView, null, null, null, null, null); Response response = dataSourcesResource.getDataSource("datasource1", "full"); ImmutableDruidDataSource result = (ImmutableDruidDataSource) response.getEntity(); Assert.assertEquals(200, response.getStatus()); @@ -329,7 +333,7 @@ public void testNullGetTheDataSource() EasyMock.replay(inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, null); + new DataSourcesResource(inventoryView, null, null, null, null, null); Assert.assertEquals(204, dataSourcesResource.getDataSource("none", null).getStatus()); EasyMock.verify(inventoryView, server); } @@ -347,7 +351,7 @@ public void testSimpleGetTheDataSource() EasyMock.replay(inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, null); + new DataSourcesResource(inventoryView, null, null, null, null, null); Response response = dataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); @@ -380,7 +384,7 @@ public void testSimpleGetTheDataSourceManyTiers() EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server, server2, server3)).atLeastOnce(); EasyMock.replay(inventoryView, server, server2, server3); - DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, null); + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, null, null); Response response = dataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); @@ -418,7 +422,7 @@ public void testSimpleGetTheDataSourceWithReplicatedSegments() EasyMock.replay(inventoryView); - DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, null); + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, null, null); Response response = dataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result1 = (Map>) response.getEntity(); @@ -463,7 +467,7 @@ public void testGetSegmentDataSourceIntervals() expectedIntervals.add(Intervals.of("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, null); + new DataSourcesResource(inventoryView, null, null, null, null, null); Response response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals( "invalidDataSource", @@ -523,7 +527,7 @@ public void testGetServedSegmentsInIntervalInDataSource() EasyMock.replay(inventoryView); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, null); + new DataSourcesResource(inventoryView, null, null, null, null, null); Response response = dataSourcesResource.getServedSegmentsInInterval( "invalidDataSource", "2010-01-01/P1D", @@ -593,7 +597,7 @@ public void testKillSegmentsInIntervalInDataSource() EasyMock.replay(indexingServiceClient, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null); + new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null, null); Response response = dataSourcesResource.killUnusedSegmentsInInterval("datasource1", interval); Assert.assertEquals(200, response.getStatus()); @@ -607,7 +611,7 @@ public void testMarkAsUnusedAllSegmentsInDataSource() IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class); EasyMock.replay(indexingServiceClient, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null); + new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null, null); try { Response response = dataSourcesResource.markAsUnusedAllSegmentsOrKillUnusedSegmentsInInterval("datasource", "true", "???"); @@ -630,7 +634,7 @@ public void testIsHandOffComplete() Rule loadRule = new IntervalLoadRule(Intervals.of("2013-01-02T00:00:00Z/2013-01-03T00:00:00Z"), null); Rule dropRule = new IntervalDropRule(Intervals.of("2013-01-01T00:00:00Z/2013-01-02T00:00:00Z")); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, databaseRuleManager, null, null); + new DataSourcesResource(inventoryView, null, databaseRuleManager, null, null, null); // test dropped EasyMock.expect(databaseRuleManager.getRulesWithDefault("dataSource1")) @@ -699,7 +703,7 @@ public void testMarkSegmentAsUsed() EasyMock.expect(segmentsMetadataManager.markSegmentAsUsed(segment.getId().toString())).andReturn(true).once(); EasyMock.replay(segmentsMetadataManager); - DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadataManager, null, null, null); + DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentAsUsed(segment.getDataSource(), segment.getId().toString()); Assert.assertEquals(200, response.getStatus()); @@ -713,7 +717,7 @@ public void testMarkSegmentAsUsedNoChange() EasyMock.expect(segmentsMetadataManager.markSegmentAsUsed(segment.getId().toString())).andReturn(false).once(); EasyMock.replay(segmentsMetadataManager); - DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadataManager, null, null, null); + DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentAsUsed(segment.getDataSource(), segment.getId().toString()); Assert.assertEquals(200, response.getStatus()); @@ -734,7 +738,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInterval() EasyMock.replay(segmentsMetadataManager, inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -757,7 +761,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsIntervalNoneUpdated() EasyMock.replay(segmentsMetadataManager, inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -780,7 +784,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsSet() throws UnknownSegmentIdsE EasyMock.replay(segmentsMetadataManager, inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -803,7 +807,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsIntervalException() EasyMock.replay(segmentsMetadataManager, inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -821,7 +825,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsNoDataSource() EasyMock.replay(segmentsMetadataManager, inventoryView, server); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -835,7 +839,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsNoDataSource() public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadNoArguments() { DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -848,7 +852,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadNoArguments() public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadBothArguments() { DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -861,7 +865,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadBothArguments() public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadEmptyArray() { DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", @@ -874,7 +878,7 @@ public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadEmptyArray() public void testMarkAsUsedNonOvershadowedSegmentsNoPayload() { DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments("datasource1", null); Assert.assertEquals(400, response.getStatus()); @@ -1026,7 +1030,7 @@ public void testMarkSegmentsAsUnused() new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(ImmutableMap.of("numChangedSegments", 1), response.getEntity()); @@ -1049,7 +1053,7 @@ public void testMarkSegmentsAsUnusedNoChanges() new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); @@ -1074,7 +1078,7 @@ public void testMarkSegmentsAsUnusedException() new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(500, response.getStatus()); Assert.assertNotNull(response.getEntity()); @@ -1096,7 +1100,7 @@ public void testMarkAsUnusedSegmentsInInterval() new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(ImmutableMap.of("numChangedSegments", 1), response.getEntity()); @@ -1119,7 +1123,7 @@ public void testMarkAsUnusedSegmentsInIntervalNoChanges() new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); @@ -1143,7 +1147,7 @@ public void testMarkAsUnusedSegmentsInIntervalException() new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(500, response.getStatus()); Assert.assertNotNull(response.getEntity()); @@ -1154,7 +1158,7 @@ public void testMarkAsUnusedSegmentsInIntervalException() public void testMarkSegmentsAsUnusedNullPayload() { DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", null); Assert.assertEquals(400, response.getStatus()); @@ -1169,7 +1173,7 @@ public void testMarkSegmentsAsUnusedNullPayload() public void testMarkSegmentsAsUnusedInvalidPayload() { DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); final DataSourcesResource.MarkDataSourceSegmentsPayload payload = new DataSourcesResource.MarkDataSourceSegmentsPayload(null, null); @@ -1183,7 +1187,7 @@ public void testMarkSegmentsAsUnusedInvalidPayload() public void testMarkSegmentsAsUnusedInvalidPayloadBothArguments() { DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null); + new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); final DataSourcesResource.MarkDataSourceSegmentsPayload payload = new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-01/P1D"), ImmutableSet.of()); @@ -1193,6 +1197,251 @@ public void testMarkSegmentsAsUnusedInvalidPayloadBothArguments() Assert.assertNotNull(response.getEntity()); } + @Test + public void testGetDatasourceLoadstatusForceMetadataRefreshNull() + { + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); + Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", null, null, null, null); + Assert.assertEquals(400, response.getStatus()); + } + + @Test + public void testGetDatasourceLoadstatusNoSegmentForInterval() + { + List segments = ImmutableList.of(); + // Test when datasource fully loaded + EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq( + "datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) + .andReturn(Optional.of(segments)).once(); + EasyMock.replay(segmentsMetadataManager); + + DataSourcesResource dataSourcesResource = new DataSourcesResource( + inventoryView, + segmentsMetadataManager, + null, + null, + null, + null + ); + Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", true, null, null, null); + Assert.assertEquals(204, response.getStatus()); + } + + @Test + public void testGetDatasourceLoadstatusDefault() + { + DataSegment datasource1Segment1 = new DataSegment( + "datasource1", + Intervals.of("2010-01-01/P1D"), + "", + null, + null, + null, + null, + 0x9, + 10 + ); + + DataSegment datasource1Segment2 = new DataSegment( + "datasource1", + Intervals.of("2010-01-22/P1D"), + "", + null, + null, + null, + null, + 0x9, + 20 + ); + DataSegment datasource2Segment1 = new DataSegment( + "datasource2", + Intervals.of("2010-01-01/P1D"), + "", + null, + null, + null, + null, + 0x9, + 30 + ); + List segments = ImmutableList.of(datasource1Segment1, datasource1Segment2); + Map completedLoadInfoMap = ImmutableMap.of( + datasource1Segment1.getId(), new SegmentLoadInfo(datasource1Segment1), + datasource1Segment2.getId(), new SegmentLoadInfo(datasource1Segment2), + datasource2Segment1.getId(), new SegmentLoadInfo(datasource2Segment1) + ); + Map halfLoadedInfoMap = ImmutableMap.of( + datasource1Segment1.getId(), new SegmentLoadInfo(datasource1Segment1) + ); + + // Test when datasource fully loaded + EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq("datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) + .andReturn(Optional.of(segments)).once(); + EasyMock.expect(inventoryView.getSegmentLoadInfos()).andReturn(completedLoadInfoMap).once(); + EasyMock.replay(segmentsMetadataManager, inventoryView); + + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); + Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", true, null, null, null); + Assert.assertEquals(200, response.getStatus()); + Assert.assertNotNull(response.getEntity()); + Assert.assertEquals(1, ((Map) response.getEntity()).size()); + Assert.assertTrue(((Map) response.getEntity()).containsKey("datasource1")); + Assert.assertEquals(100.0, ((Map) response.getEntity()).get("datasource1")); + EasyMock.verify(segmentsMetadataManager, inventoryView); + EasyMock.reset(segmentsMetadataManager, inventoryView); + + // Test when datasource half loaded + EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq("datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) + .andReturn(Optional.of(segments)).once(); + EasyMock.expect(inventoryView.getSegmentLoadInfos()).andReturn(halfLoadedInfoMap).once(); + EasyMock.replay(segmentsMetadataManager, inventoryView); + + dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); + response = dataSourcesResource.getDatasourceLoadstatus("datasource1", true, null, null, null); + Assert.assertEquals(200, response.getStatus()); + Assert.assertNotNull(response.getEntity()); + Assert.assertEquals(1, ((Map) response.getEntity()).size()); + Assert.assertTrue(((Map) response.getEntity()).containsKey("datasource1")); + Assert.assertEquals(50.0, ((Map) response.getEntity()).get("datasource1")); + EasyMock.verify(segmentsMetadataManager, inventoryView); + } + + @Test + public void testGetDatasourceLoadstatusSimple() + { + DataSegment datasource1Segment1 = new DataSegment( + "datasource1", + Intervals.of("2010-01-01/P1D"), + "", + null, + null, + null, + null, + 0x9, + 10 + ); + + DataSegment datasource1Segment2 = new DataSegment( + "datasource1", + Intervals.of("2010-01-22/P1D"), + "", + null, + null, + null, + null, + 0x9, + 20 + ); + DataSegment datasource2Segment1 = new DataSegment( + "datasource2", + Intervals.of("2010-01-01/P1D"), + "", + null, + null, + null, + null, + 0x9, + 30 + ); + List segments = ImmutableList.of(datasource1Segment1, datasource1Segment2); + Map completedLoadInfoMap = ImmutableMap.of( + datasource1Segment1.getId(), new SegmentLoadInfo(datasource1Segment1), + datasource1Segment2.getId(), new SegmentLoadInfo(datasource1Segment2), + datasource2Segment1.getId(), new SegmentLoadInfo(datasource2Segment1) + ); + Map halfLoadedInfoMap = ImmutableMap.of( + datasource1Segment1.getId(), new SegmentLoadInfo(datasource1Segment1) + ); + + // Test when datasource fully loaded + EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq("datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) + .andReturn(Optional.of(segments)).once(); + EasyMock.expect(inventoryView.getSegmentLoadInfos()).andReturn(completedLoadInfoMap).once(); + EasyMock.replay(segmentsMetadataManager, inventoryView); + + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); + Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", true, null, "simple", null); + Assert.assertEquals(200, response.getStatus()); + Assert.assertNotNull(response.getEntity()); + Assert.assertEquals(1, ((Map) response.getEntity()).size()); + Assert.assertTrue(((Map) response.getEntity()).containsKey("datasource1")); + Assert.assertEquals(0, ((Map) response.getEntity()).get("datasource1")); + EasyMock.verify(segmentsMetadataManager, inventoryView); + EasyMock.reset(segmentsMetadataManager, inventoryView); + + // Test when datasource half loaded + EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq("datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) + .andReturn(Optional.of(segments)).once(); + EasyMock.expect(inventoryView.getSegmentLoadInfos()).andReturn(halfLoadedInfoMap).once(); + EasyMock.replay(segmentsMetadataManager, inventoryView); + + dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, null); + response = dataSourcesResource.getDatasourceLoadstatus("datasource1", true, null, "simple", null); + Assert.assertEquals(200, response.getStatus()); + Assert.assertNotNull(response.getEntity()); + Assert.assertEquals(1, ((Map) response.getEntity()).size()); + Assert.assertTrue(((Map) response.getEntity()).containsKey("datasource1")); + Assert.assertEquals(1, ((Map) response.getEntity()).get("datasource1")); + EasyMock.verify(segmentsMetadataManager, inventoryView); + } + + @Test + public void testGetDatasourceLoadstatusFull() + { + DataSegment datasource1Segment1 = new DataSegment( + "datasource1", + Intervals.of("2010-01-01/P1D"), + "", + null, + null, + null, + null, + 0x9, + 10 + ); + + DataSegment datasource1Segment2 = new DataSegment( + "datasource1", + Intervals.of("2010-01-22/P1D"), + "", + null, + null, + null, + null, + 0x9, + 20 + ); + List segments = ImmutableList.of(datasource1Segment1, datasource1Segment2); + + final Map> underReplicationCountsPerDataSourcePerTier = new HashMap<>(); + Object2LongMap tier1 = new Object2LongOpenHashMap<>(); + tier1.put("datasource1", 0L); + Object2LongMap tier2 = new Object2LongOpenHashMap<>(); + tier2.put("datasource1", 3L); + underReplicationCountsPerDataSourcePerTier.put("tier1", tier1); + underReplicationCountsPerDataSourcePerTier.put("tier2", tier2); + + // Test when datasource fully loaded + EasyMock.expect(segmentsMetadataManager.iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(EasyMock.eq("datasource1"), EasyMock.anyObject(Interval.class), EasyMock.anyBoolean())) + .andReturn(Optional.of(segments)).once(); + DruidCoordinator druidCoordinator = EasyMock.createMock(DruidCoordinator.class); + EasyMock.expect(druidCoordinator.computeUnderReplicationCountsPerDataSourcePerTierForSegments(segments)) + .andReturn(underReplicationCountsPerDataSourcePerTier).once(); + + EasyMock.replay(segmentsMetadataManager, druidCoordinator); + + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, segmentsMetadataManager, null, null, null, druidCoordinator); + Response response = dataSourcesResource.getDatasourceLoadstatus("datasource1", true, null, null, "full"); + Assert.assertEquals(200, response.getStatus()); + Assert.assertNotNull(response.getEntity()); + Assert.assertEquals(2, ((Map) response.getEntity()).size()); + Assert.assertEquals(1, ((Map) ((Map) response.getEntity()).get("tier1")).size()); + Assert.assertEquals(1, ((Map) ((Map) response.getEntity()).get("tier2")).size()); + Assert.assertEquals(0L, ((Map) ((Map) response.getEntity()).get("tier1")).get("datasource1")); + Assert.assertEquals(3L, ((Map) ((Map) response.getEntity()).get("tier2")).get("datasource1")); + EasyMock.verify(segmentsMetadataManager); + } + private DruidServerMetadata createRealtimeServerMetadata(String name) { return createServerMetadata(name, ServerType.REALTIME); diff --git a/server/src/test/java/org/apache/druid/server/log/FilteredRequestLoggerTest.java b/server/src/test/java/org/apache/druid/server/log/FilteredRequestLoggerTest.java index 75f4473c1601..54ee6e2a8f88 100644 --- a/server/src/test/java/org/apache/druid/server/log/FilteredRequestLoggerTest.java +++ b/server/src/test/java/org/apache/druid/server/log/FilteredRequestLoggerTest.java @@ -26,8 +26,8 @@ import com.google.inject.ProvisionException; import org.apache.druid.guice.JsonConfigurator; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.query.LegacyDataSource; import org.apache.druid.query.Query; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery; import org.apache.druid.server.QueryStats; import org.apache.druid.server.RequestLogLine; @@ -47,7 +47,7 @@ public class FilteredRequestLoggerTest public final ExpectedException expectedException = ExpectedException.none(); private final DefaultObjectMapper mapper = new DefaultObjectMapper(); private final SegmentMetadataQuery testSegmentMetadataQuery = new SegmentMetadataQuery( - new LegacyDataSource("foo"), + new TableDataSource("foo"), null, null, null, diff --git a/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java b/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java index ed56572e3d35..87b02327dcb9 100644 --- a/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java +++ b/server/src/test/java/org/apache/druid/server/log/LoggingRequestLoggerTest.java @@ -29,11 +29,11 @@ import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.DataSource; -import org.apache.druid.query.LegacyDataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.UnionDataSource; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.spec.QuerySegmentSpec; @@ -73,7 +73,7 @@ public class LoggingRequestLoggerTest final String remoteAddr = "some.host.tld"; final Map queryContext = ImmutableMap.of("foo", "bar"); final Query query = new FakeQuery( - new LegacyDataSource("datasource"), + new TableDataSource("datasource"), new QuerySegmentSpec() { @Override @@ -127,7 +127,7 @@ public QueryRunner lookup(Query query, QuerySegmentWalker walker) ); final Query unionQuery = new FakeQuery( - new UnionDataSource(ImmutableList.of(new LegacyDataSource("A"), new LegacyDataSource("B"))), + new UnionDataSource(ImmutableList.of(new TableDataSource("A"), new TableDataSource("B"))), new QuerySegmentSpec() { @Override diff --git a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java index 962c950e7ee8..caa93cad87e6 100644 --- a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java +++ b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupCoordinatorManagerTest.java @@ -108,7 +108,7 @@ public class LookupCoordinatorManagerTest private static final LookupsState LOOKUPS_STATE = new LookupsState<>( SINGLE_LOOKUP_MAP_V0, SINGLE_LOOKUP_MAP_V1, - Collections.EMPTY_SET + Collections.emptySet() ); private static final AtomicLong EVENT_EMITS = new AtomicLong(0L); @@ -1352,7 +1352,7 @@ public void testMultipleStartStop() throws Exception EasyMock.anyObject(), EasyMock.isNull() )).andReturn( - new AtomicReference<>(Collections.EMPTY_MAP)).anyTimes(); + new AtomicReference<>(Collections.emptyMap())).anyTimes(); EasyMock.replay(configManager); diff --git a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java index bfd72b4b8851..e94b8068424e 100644 --- a/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/server/shard/NumberedShardSpecTest.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Ordering; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.ServerTestHelper; import org.apache.druid.timeline.Overshadowable; @@ -45,6 +46,12 @@ public class NumberedShardSpecTest { + @Test + public void testEquals() + { + EqualsVerifier.forClass(NumberedShardSpec.class).usingGetClass().verify(); + } + @Test public void testSerdeRoundTrip() throws Exception { @@ -53,7 +60,7 @@ public void testSerdeRoundTrip() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) spec).getNumCorePartitions()); } @Test @@ -64,7 +71,7 @@ public void testSerdeBackwardsCompat() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((NumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, ((NumberedShardSpec) spec).getNumCorePartitions()); } @Test @@ -128,29 +135,29 @@ public void testVersionedIntervalTimelineBehaviorForNumberedShardSpec() //incomplete partition sets testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk0), - Collections.EMPTY_SET + Collections.emptySet() ); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk1), - Collections.EMPTY_SET + Collections.emptySet() ); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk4), - Collections.EMPTY_SET + Collections.emptySet() ); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk0, chunk4), - Collections.EMPTY_SET + Collections.emptySet() ); testVersionedIntervalTimelineBehaviorForNumberedShardSpec( ImmutableList.of(chunk1, chunk4), - Collections.EMPTY_SET + Collections.emptySet() ); //complete partition sets diff --git a/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java b/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java index 2722179973e7..0787cf2c119c 100644 --- a/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/server/shard/SingleDimensionShardSpecTest.java @@ -158,7 +158,7 @@ private SingleDimensionShardSpec makeSpec(String start, String end) private SingleDimensionShardSpec makeSpec(String dimension, String start, String end) { - return new SingleDimensionShardSpec(dimension, start, end, 0); + return new SingleDimensionShardSpec(dimension, start, end, 0, SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS); } private Map makeMap(String value) diff --git a/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java b/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java index 9ac4d27823d8..ecb110237498 100644 --- a/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java +++ b/server/src/test/java/org/apache/druid/timeline/partition/HashBasedNumberedShardSpecTest.java @@ -19,10 +19,12 @@ package org.apache.druid.timeline.partition; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.data.input.InputRow; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.Row; @@ -41,6 +43,16 @@ public class HashBasedNumberedShardSpecTest { + @Test + public void testEquals() + { + EqualsVerifier.forClass(HashBasedNumberedShardSpec.class) + .withIgnoredFields("jsonMapper") + .withPrefabValues(ObjectMapper.class, new ObjectMapper(), new ObjectMapper()) + .usingGetClass() + .verify(); + } + @Test public void testSerdeRoundTrip() throws Exception { @@ -50,6 +62,8 @@ public void testSerdeRoundTrip() throws Exception new HashBasedNumberedShardSpec( 1, 2, + 1, + 3, ImmutableList.of("visitor_id"), ServerTestHelper.MAPPER ) @@ -57,7 +71,9 @@ public void testSerdeRoundTrip() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, spec.getNumCorePartitions()); + Assert.assertEquals(1, ((HashBasedNumberedShardSpec) spec).getBucketId()); + Assert.assertEquals(3, ((HashBasedNumberedShardSpec) spec).getNumBuckets()); Assert.assertEquals(ImmutableList.of("visitor_id"), ((HashBasedNumberedShardSpec) spec).getPartitionDimensions()); } @@ -69,24 +85,28 @@ public void testSerdeBackwardsCompat() throws Exception ShardSpec.class ); Assert.assertEquals(1, spec.getPartitionNum()); - Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getPartitions()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) spec).getNumCorePartitions()); final ShardSpec specWithPartitionDimensions = ServerTestHelper.MAPPER.readValue( "{\"type\": \"hashed\", \"partitions\": 2, \"partitionNum\": 1, \"partitionDimensions\":[\"visitor_id\"]}", ShardSpec.class ); Assert.assertEquals(1, specWithPartitionDimensions.getPartitionNum()); - Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getPartitions()); - Assert.assertEquals(ImmutableList.of("visitor_id"), ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getPartitionDimensions()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getNumCorePartitions()); + Assert.assertEquals(2, ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getNumBuckets()); + Assert.assertEquals( + ImmutableList.of("visitor_id"), + ((HashBasedNumberedShardSpec) specWithPartitionDimensions).getPartitionDimensions() + ); } @Test public void testPartitionChunks() { final List specs = ImmutableList.of( - new HashBasedNumberedShardSpec(0, 3, null, ServerTestHelper.MAPPER), - new HashBasedNumberedShardSpec(1, 3, null, ServerTestHelper.MAPPER), - new HashBasedNumberedShardSpec(2, 3, null, ServerTestHelper.MAPPER) + new HashBasedNumberedShardSpec(0, 3, 0, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(1, 3, 1, 3, null, ServerTestHelper.MAPPER), + new HashBasedNumberedShardSpec(2, 3, 2, 3, null, ServerTestHelper.MAPPER) ); final List> chunks = Lists.transform( @@ -157,35 +177,26 @@ public void testIsInChunkWithMorePartitionsBeyondNumBucketsReturningTrue() @Test public void testGetGroupKey() { - final HashBasedNumberedShardSpec shardSpec1 = new HashBasedNumberedShardSpec( - 1, - 2, - ImmutableList.of("visitor_id"), - ServerTestHelper.MAPPER - ); + final List partitionDimensions1 = ImmutableList.of("visitor_id"); final DateTime time = DateTimes.nowUtc(); final InputRow inputRow = new MapBasedInputRow( time, ImmutableList.of("visitor_id", "cnt"), ImmutableMap.of("visitor_id", "v1", "cnt", 10) ); - Assert.assertEquals(ImmutableList.of(Collections.singletonList("v1")), shardSpec1.getGroupKey(time.getMillis(), inputRow)); - - final HashBasedNumberedShardSpec shardSpec2 = new HashBasedNumberedShardSpec( - 1, - 2, - null, - ServerTestHelper.MAPPER + Assert.assertEquals( + ImmutableList.of(Collections.singletonList("v1")), + HashBasedNumberedShardSpec.getGroupKey(partitionDimensions1, time.getMillis(), inputRow) ); - Assert.assertEquals(ImmutableList.of( + + Assert.assertEquals( + ImmutableList.of( time.getMillis(), - ImmutableMap.of( - "cnt", - Collections.singletonList(10), - "visitor_id", - Collections.singletonList("v1") - ) - ).toString(), shardSpec2.getGroupKey(time.getMillis(), inputRow).toString()); + ImmutableMap.of("cnt", Collections.singletonList(10), "visitor_id", Collections.singletonList("v1"))) + .toString(), + // empty list when partitionDimensions is null + HashBasedNumberedShardSpec.getGroupKey(ImmutableList.of(), time.getMillis(), inputRow).toString() + ); } public boolean assertExistsInOneSpec(List specs, InputRow row) @@ -202,7 +213,7 @@ public static class HashOverridenShardSpec extends HashBasedNumberedShardSpec { public HashOverridenShardSpec(int partitionNum, int partitions) { - super(partitionNum, partitions, null, ServerTestHelper.MAPPER); + super(partitionNum, partitions, partitionNum, partitions, null, ServerTestHelper.MAPPER); } @Override diff --git a/services/src/main/java/org/apache/druid/cli/CliBroker.java b/services/src/main/java/org/apache/druid/cli/CliBroker.java index 5badcb0e0e3a..18d2813fa411 100644 --- a/services/src/main/java/org/apache/druid/cli/CliBroker.java +++ b/services/src/main/java/org/apache/druid/cli/CliBroker.java @@ -33,6 +33,7 @@ import org.apache.druid.client.selector.CustomTierSelectorStrategyConfig; import org.apache.druid.client.selector.ServerSelectorStrategy; import org.apache.druid.client.selector.TierSelectorStrategy; +import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.CacheModule; @@ -42,9 +43,11 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.SegmentWranglerModule; +import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.RetryQueryRunnerConfig; @@ -52,7 +55,12 @@ import org.apache.druid.server.BrokerQueryResource; import org.apache.druid.server.ClientInfoResource; import org.apache.druid.server.ClientQuerySegmentWalker; +import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordination.ZkCoordinator; import org.apache.druid.server.http.BrokerResource; +import org.apache.druid.server.http.HistoricalResource; +import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.QueryCountStatsProvider; @@ -123,12 +131,19 @@ protected List getModules() Jerseys.addResource(binder, HttpServerInventoryViewResource.class); LifecycleModule.register(binder, Server.class); + binder.bind(SegmentManager.class).in(LazySingleton.class); + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.BROKER)); + Jerseys.addResource(binder, HistoricalResource.class); + Jerseys.addResource(binder, SegmentListerResource.class); + + LifecycleModule.register(binder, ZkCoordinator.class); bindNodeRoleAndAnnouncer( binder, DiscoverySideEffectsProvider .builder(NodeRole.BROKER) - .serviceClasses(ImmutableList.of(LookupNodeService.class)) + .serviceClasses(ImmutableList.of(DataNodeService.class, LookupNodeService.class)) .useLegacyAnnouncer(true) .build() ); diff --git a/services/src/main/java/org/apache/druid/cli/CliIndexer.java b/services/src/main/java/org/apache/druid/cli/CliIndexer.java index 7483fec93a93..7029d6f039bc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliIndexer.java @@ -25,9 +25,9 @@ import com.google.inject.Module; import com.google.inject.Provides; import com.google.inject.name.Names; -import com.google.inject.util.Providers; import io.airlift.airline.Command; import org.apache.druid.client.DruidServer; +import org.apache.druid.client.DruidServerConfig; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.LookupNodeService; import org.apache.druid.discovery.NodeRole; @@ -43,6 +43,7 @@ import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; +import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; @@ -58,14 +59,16 @@ import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.indexing.worker.http.ShuffleResource; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; -import org.apache.druid.segment.realtime.CliIndexerDataSegmentServerAnnouncerLifecycleHandler; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.UnifiedIndexerAppenderatorsManager; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.SegmentLoadDropHandler; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordination.ZkCoordinator; +import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.initialization.jetty.CliIndexerServerModule; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -138,14 +141,14 @@ public void configure(Binder binder) binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class); Jerseys.addResource(binder, SegmentListerResource.class); - - LifecycleModule.register(binder, CliIndexerDataSegmentServerAnnouncerLifecycleHandler.class); - Jerseys.addResource(binder, ShuffleResource.class); LifecycleModule.register(binder, Server.class, RemoteChatHandler.class); - binder.bind(SegmentLoadDropHandler.class).toProvider(Providers.of(null)); + binder.bind(SegmentManager.class).in(LazySingleton.class); + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + Jerseys.addResource(binder, HistoricalResource.class); + LifecycleModule.register(binder, ZkCoordinator.class); bindNodeRoleAndAnnouncer( binder, @@ -186,11 +189,11 @@ public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) @Provides @LazySingleton - public DataNodeService getDataNodeService() + public DataNodeService getDataNodeService(DruidServerConfig serverConfig) { return new DataNodeService( DruidServer.DEFAULT_TIER, - 0L, + serverConfig.getMaxSize(), ServerType.INDEXER_EXECUTOR, DruidServer.DEFAULT_PRIORITY ); @@ -200,6 +203,7 @@ public DataNodeService getDataNodeService() new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new QueryablePeonModule(), new CliIndexerServerModule(properties), new LookupModule() diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index d273c16ad331..8b3d12f286eb 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -61,6 +61,7 @@ import org.apache.druid.indexing.worker.http.TaskManagementResource; import org.apache.druid.indexing.worker.http.WorkerResource; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; @@ -186,6 +187,7 @@ public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new LookupSerdeModule() ); } diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 21ffa9a8bff2..4baf155d229c 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -98,6 +98,7 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorResource; import org.apache.druid.indexing.worker.config.WorkerConfig; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.lookup.LookupSerdeModule; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; @@ -347,6 +348,7 @@ private void configureOverlordHelpers(Binder binder) new IndexingServiceInputSourceModule(), new IndexingServiceTaskLogsModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new SupervisorModule(), new LookupSerdeModule(), new SamplerModule() 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 1160eb9240c1..2eb2c68521fc 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -61,7 +61,6 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Parent; import org.apache.druid.guice.annotations.Self; -import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; @@ -92,6 +91,7 @@ import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; +import org.apache.druid.metadata.input.InputSourceModule; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; import org.apache.druid.segment.loading.DataSegmentArchiver; @@ -109,15 +109,16 @@ import org.apache.druid.segment.realtime.plumber.CoordinatorBasedSegmentHandoffNotifierFactory; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import org.apache.druid.server.DruidNode; -import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordination.ZkCoordinator; +import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; import org.apache.druid.server.initialization.jetty.ChatHandlerServerModule; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.eclipse.jetty.server.Server; -import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.List; @@ -154,6 +155,14 @@ public class CliPeon extends GuiceRunnable @Option(name = "--nodeType", title = "nodeType", description = "Set the node type to expose on ZK") public String serverType = "indexer-executor"; + + /** + * 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") + public String loadBroadcastSegments = "false"; + private static final Logger log = new Logger(CliPeon.class); @Inject @@ -174,6 +183,7 @@ protected List getModules() new JoinableFactoryModule(), new Module() { + @SuppressForbidden(reason = "System#out, System#err") @Override public void configure(Binder binder) { @@ -218,6 +228,13 @@ public void configure(Binder binder) Jerseys.addResource(binder, SegmentListerResource.class); binder.bind(ServerTypeConfig.class).toInstance(new ServerTypeConfig(ServerType.fromString(serverType))); LifecycleModule.register(binder, Server.class); + + if ("true".equals(loadBroadcastSegments)) { + binder.bind(SegmentManager.class).in(LazySingleton.class); + binder.bind(ZkCoordinator.class).in(ManageLifecycle.class); + Jerseys.addResource(binder, HistoricalResource.class); + LifecycleModule.register(binder, ZkCoordinator.class); + } } @Provides @@ -247,21 +264,12 @@ public String getTaskIDFromTask(final Task task) { return task.getId(); } - - @Provides - public SegmentListerResource getSegmentListerResource( - @Json ObjectMapper jsonMapper, - @Smile ObjectMapper smileMapper, - @Nullable BatchDataSegmentAnnouncer announcer - ) - { - return new SegmentListerResource(jsonMapper, smileMapper, announcer, null); - } }, new QueryablePeonModule(), new IndexingServiceFirehoseModule(), new IndexingServiceInputSourceModule(), new IndexingServiceTuningConfigModule(), + new InputSourceModule(), new ChatHandlerServerModule(properties), new LookupModule() ); diff --git a/services/src/test/java/org/apache/druid/cli/MainTest.java b/services/src/test/java/org/apache/druid/cli/MainTest.java index 3e960f69e442..a1fbed581e5d 100644 --- a/services/src/test/java/org/apache/druid/cli/MainTest.java +++ b/services/src/test/java/org/apache/druid/cli/MainTest.java @@ -50,7 +50,9 @@ public static Iterable constructorFeeder() //new Object[]{new CliInternalHadoopIndexer()}, new Object[]{new CliMiddleManager()}, - new Object[]{new CliRouter()} + new Object[]{new CliRouter()}, + + new Object[]{new CliIndexer()} ); } diff --git a/setup-hooks.sh b/setup-hooks.sh new file mode 100755 index 000000000000..85c60d1a7250 --- /dev/null +++ b/setup-hooks.sh @@ -0,0 +1,17 @@ +#!/bin/bash -eu +# 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. + +ln -s ../../hooks/pre-push.sh .git/hooks/pre-push diff --git a/sql/pom.xml b/sql/pom.xml index ef8d3f79cf64..cd1c12e9ef97 100644 --- a/sql/pom.xml +++ b/sql/pom.xml @@ -184,6 +184,11 @@ junit test + + pl.pragmatists + JUnitParams + test + org.apache.calcite calcite-core diff --git a/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java b/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java index d5b27aad6489..738bed4dba7d 100644 --- a/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java +++ b/sql/src/main/java/org/apache/druid/sql/avatica/DruidMeta.java @@ -364,7 +364,7 @@ public MetaResultSet getSchemas( } if (schemaPattern.s != null) { - whereBuilder.add("SCHEMATA.SCHEMA_NAME LIKE " + Calcites.escapeStringLiteral(schemaPattern.s)); + whereBuilder.add("SCHEMATA.SCHEMA_NAME LIKE " + withEscapeClause(schemaPattern.s)); } final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder); @@ -395,11 +395,11 @@ public MetaResultSet getTables( } if (schemaPattern.s != null) { - whereBuilder.add("TABLES.TABLE_SCHEMA LIKE " + Calcites.escapeStringLiteral(schemaPattern.s)); + whereBuilder.add("TABLES.TABLE_SCHEMA LIKE " + withEscapeClause(schemaPattern.s)); } if (tableNamePattern.s != null) { - whereBuilder.add("TABLES.TABLE_NAME LIKE " + Calcites.escapeStringLiteral(tableNamePattern.s)); + whereBuilder.add("TABLES.TABLE_NAME LIKE " + withEscapeClause(tableNamePattern.s)); } if (typeList != null) { @@ -446,15 +446,16 @@ public MetaResultSet getColumns( } if (schemaPattern.s != null) { - whereBuilder.add("COLUMNS.TABLE_SCHEMA LIKE " + Calcites.escapeStringLiteral(schemaPattern.s)); + whereBuilder.add("COLUMNS.TABLE_SCHEMA LIKE " + withEscapeClause(schemaPattern.s)); } if (tableNamePattern.s != null) { - whereBuilder.add("COLUMNS.TABLE_NAME LIKE " + Calcites.escapeStringLiteral(tableNamePattern.s)); + whereBuilder.add("COLUMNS.TABLE_NAME LIKE " + withEscapeClause(tableNamePattern.s)); } if (columnNamePattern.s != null) { - whereBuilder.add("COLUMNS.COLUMN_NAME LIKE " + Calcites.escapeStringLiteral(columnNamePattern.s)); + whereBuilder.add("COLUMNS.COLUMN_NAME LIKE " + + withEscapeClause(columnNamePattern.s)); } final String where = whereBuilder.isEmpty() ? "" : "WHERE " + Joiner.on(" AND ").join(whereBuilder); @@ -638,4 +639,9 @@ private int getEffectiveMaxRowsPerFrame(int clientMaxRowsPerFrame) } return Math.min(clientMaxRowsPerFrame, config.getMaxRowsPerFrame()); } + + private static String withEscapeClause(String toEscape) + { + return Calcites.escapeStringLiteral(toEscape) + " ESCAPE '\\'"; + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java index 6b4a4539bee5..23c48df601b1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/OperatorConversions.java @@ -24,11 +24,13 @@ import com.google.common.collect.Iterables; import it.unimi.dsi.fastutil.ints.IntArraySet; import it.unimi.dsi.fastutil.ints.IntSet; +import it.unimi.dsi.fastutil.ints.IntSets; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexLiteral; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.runtime.CalciteException; import org.apache.calcite.sql.SqlCallBinding; import org.apache.calcite.sql.SqlFunction; import org.apache.calcite.sql.SqlFunctionCategory; @@ -222,6 +224,10 @@ public static PostAggregator toPostAggregator( } } + /** + * Returns a builder that helps {@link SqlOperatorConversion} implementations create the {@link SqlFunction} + * objects they need to return from {@link SqlOperatorConversion#calciteOperator()}. + */ public static OperatorBuilder operatorBuilder(final String name) { return new OperatorBuilder(name); @@ -238,6 +244,7 @@ public static class OperatorBuilder private SqlOperandTypeChecker operandTypeChecker; private List operandTypes; private Integer requiredOperands = null; + private int[] literalOperands = null; private SqlOperandTypeInference operandTypeInference; private OperatorBuilder(final String name) @@ -245,64 +252,123 @@ private OperatorBuilder(final String name) this.name = Preconditions.checkNotNull(name, "name"); } - public OperatorBuilder kind(final SqlKind kind) + /** + * Sets the return type of the operator to "typeName", marked as non-nullable. + * + * One of {@link #returnTypeNonNull}, {@link #returnTypeNullable}, or + * {@link #returnTypeInference(SqlReturnTypeInference)} must be used before calling {@link #build()}. These methods + * cannot be mixed; you must call exactly one. + */ + public OperatorBuilder returnTypeNonNull(final SqlTypeName typeName) { - this.kind = kind; - return this; - } + Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); - public OperatorBuilder returnType(final SqlTypeName typeName) - { this.returnTypeInference = ReturnTypes.explicit( factory -> Calcites.createSqlType(factory, typeName) ); return this; } - public OperatorBuilder nullableReturnType(final SqlTypeName typeName) + /** + * Sets the return type of the operator to "typeName", marked as nullable. + * + * One of {@link #returnTypeNonNull}, {@link #returnTypeNullable}, or + * {@link #returnTypeInference(SqlReturnTypeInference)} must be used before calling {@link #build()}. These methods + * cannot be mixed; you must call exactly one. + */ + public OperatorBuilder returnTypeNullable(final SqlTypeName typeName) { + Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); + this.returnTypeInference = ReturnTypes.explicit( factory -> Calcites.createSqlTypeWithNullability(factory, typeName, true) ); return this; } + /** + * Provides customized return type inference logic. + * + * One of {@link #returnTypeNonNull}, {@link #returnTypeNullable}, or + * {@link #returnTypeInference(SqlReturnTypeInference)} must be used before calling {@link #build()}. These methods + * cannot be mixed; you must call exactly one. + */ public OperatorBuilder returnTypeInference(final SqlReturnTypeInference returnTypeInference) { + Preconditions.checkState(this.returnTypeInference == null, "Cannot set return type multiple times"); + this.returnTypeInference = returnTypeInference; return this; } + /** + * Sets the {@link SqlKind} of the operator. + * + * The default, if not provided, is {@link SqlFunctionCategory#USER_DEFINED_FUNCTION}. + */ public OperatorBuilder functionCategory(final SqlFunctionCategory functionCategory) { this.functionCategory = functionCategory; return this; } + /** + * Provides customized operand type checking logic. + * + * One of {@link #operandTypes(SqlTypeFamily...)} or {@link #operandTypeChecker(SqlOperandTypeChecker)} must be used + * before calling {@link #build()}. These methods cannot be mixed; you must call exactly one. + */ public OperatorBuilder operandTypeChecker(final SqlOperandTypeChecker operandTypeChecker) { this.operandTypeChecker = operandTypeChecker; return this; } + /** + * Signifies that a function accepts operands of type family given by {@param operandTypes}. + * + * May be used in conjunction with {@link #requiredOperands(int)} and {@link #literalOperands(int...)} in order + * to further refine operand checking logic. + * + * For deeper control, use {@link #operandTypeChecker(SqlOperandTypeChecker)} instead. + */ public OperatorBuilder operandTypes(final SqlTypeFamily... operandTypes) { this.operandTypes = Arrays.asList(operandTypes); return this; } - public OperatorBuilder operandTypeInference(final SqlOperandTypeInference operandTypeInference) + /** + * Signifies that the first {@code requiredOperands} operands are required, and all later operands are optional. + * + * Required operands are not allowed to be null. Optional operands can either be skipped or explicitly provided as + * literal NULLs. For example, if {@code requiredOperands == 1}, then {@code F(x, NULL)} and {@code F(x)} are both + * accepted, and {@code x} must not be null. + * + * Must be used in conjunction with {@link #operandTypes(SqlTypeFamily...)}; this method is not compatible with + * {@link #operandTypeChecker(SqlOperandTypeChecker)}. + */ + public OperatorBuilder requiredOperands(final int requiredOperands) { - this.operandTypeInference = operandTypeInference; + this.requiredOperands = requiredOperands; return this; } - public OperatorBuilder requiredOperands(final int requiredOperands) + /** + * Signifies that the operands at positions given by {@code literalOperands} must be literals. + * + * Must be used in conjunction with {@link #operandTypes(SqlTypeFamily...)}; this method is not compatible with + * {@link #operandTypeChecker(SqlOperandTypeChecker)}. + */ + public OperatorBuilder literalOperands(final int... literalOperands) { - this.requiredOperands = requiredOperands; + this.literalOperands = literalOperands; return this; } + /** + * Creates a {@link SqlFunction} from this builder. + */ public SqlFunction build() { // Create "nullableOperands" set including all optional arguments. @@ -317,13 +383,14 @@ public SqlFunction build() theOperandTypeChecker = new DefaultOperandTypeChecker( operandTypes, requiredOperands == null ? operandTypes.size() : requiredOperands, - nullableOperands + nullableOperands, + literalOperands ); - } else if (operandTypes == null && requiredOperands == null) { + } else if (operandTypes == null && requiredOperands == null && literalOperands == null) { theOperandTypeChecker = operandTypeChecker; } else { throw new ISE( - "Cannot have both 'operandTypeChecker' and 'operandTypes' / 'requiredOperands'" + "Cannot have both 'operandTypeChecker' and 'operandTypes' / 'requiredOperands' / 'literalOperands'" ); } @@ -430,36 +497,56 @@ public void inferOperandTypes( /** * Operand type checker that is used in 'simple' situations: there are a particular number of operands, with - * particular types, some of which may be optional or nullable. + * particular types, some of which may be optional or nullable, and some of which may be required to be literals. */ private static class DefaultOperandTypeChecker implements SqlOperandTypeChecker { private final List operandTypes; private final int requiredOperands; private final IntSet nullableOperands; + private final IntSet literalOperands; DefaultOperandTypeChecker( final List operandTypes, final int requiredOperands, - final IntSet nullableOperands + final IntSet nullableOperands, + @Nullable final int[] literalOperands ) { Preconditions.checkArgument(requiredOperands <= operandTypes.size() && requiredOperands >= 0); this.operandTypes = Preconditions.checkNotNull(operandTypes, "operandTypes"); this.requiredOperands = requiredOperands; this.nullableOperands = Preconditions.checkNotNull(nullableOperands, "nullableOperands"); + + if (literalOperands == null) { + this.literalOperands = IntSets.EMPTY_SET; + } else { + this.literalOperands = new IntArraySet(); + Arrays.stream(literalOperands).forEach(this.literalOperands::add); + } } @Override public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) { - if (operandTypes.size() != callBinding.getOperandCount()) { - // Just like FamilyOperandTypeChecker: assume this is an inapplicable sub-rule of a composite rule; don't throw - return false; - } - for (int i = 0; i < callBinding.operands().size(); i++) { final SqlNode operand = callBinding.operands().get(i); + + if (literalOperands.contains(i)) { + // Verify that 'operand' is a literal. + if (!SqlUtil.isLiteral(operand)) { + return throwOrReturn( + throwOnFailure, + callBinding, + cb -> cb.getValidator() + .newValidationError( + operand, + Static.RESOURCE.argumentMustBeLiteral(callBinding.getOperator().getName()) + ) + ); + } + } + final RelDataType operandType = callBinding.getValidator().deriveType(callBinding.getScope(), operand); final SqlTypeFamily expectedFamily = operandTypes.get(i); @@ -467,21 +554,21 @@ public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFail // ANY matches anything. This operand is all good; do nothing. } else if (expectedFamily.getTypeNames().contains(operandType.getSqlTypeName())) { // Operand came in with one of the expected types. - } else if (operandType.getSqlTypeName() == SqlTypeName.NULL) { + } else if (operandType.getSqlTypeName() == SqlTypeName.NULL || SqlUtil.isNullLiteral(operand, true)) { // Null came in, check if operand is a nullable type. if (!nullableOperands.contains(i)) { - if (throwOnFailure) { - throw callBinding.getValidator().newValidationError(operand, Static.RESOURCE.nullIllegal()); - } else { - return false; - } + return throwOrReturn( + throwOnFailure, + callBinding, + cb -> cb.getValidator().newValidationError(operand, Static.RESOURCE.nullIllegal()) + ); } } else { - if (throwOnFailure) { - throw callBinding.newValidationSignatureError(); - } else { - return false; - } + return throwOrReturn( + throwOnFailure, + callBinding, + SqlCallBinding::newValidationSignatureError + ); } } @@ -512,4 +599,17 @@ public boolean isOptional(int i) return i + 1 > requiredOperands; } } + + private static boolean throwOrReturn( + final boolean throwOnFailure, + final SqlCallBinding callBinding, + final Function exceptionMapper + ) + { + if (throwOnFailure) { + throw exceptionMapper.apply(callBinding); + } else { + return false; + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayLengthOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayLengthOperatorConversion.java index bcafbb605c69..3bcd6ec0de06 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayLengthOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayLengthOperatorConversion.java @@ -44,7 +44,7 @@ public class ArrayLengthOperatorConversion implements SqlOperatorConversion ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.INTEGER) + .returnTypeNonNull(SqlTypeName.INTEGER) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOfOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOfOperatorConversion.java index 2e06ef9aa290..8d8da7461bb0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOfOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOfOperatorConversion.java @@ -48,7 +48,7 @@ public class ArrayOffsetOfOperatorConversion implements SqlOperatorConversion ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.INTEGER) + .returnTypeNonNull(SqlTypeName.INTEGER) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOperatorConversion.java index 42a874da5e45..2fdee5d79b24 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOffsetOperatorConversion.java @@ -48,7 +48,7 @@ public class ArrayOffsetOperatorConversion implements SqlOperatorConversion ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOfOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOfOperatorConversion.java index 367d80ff1cc4..14fa9d61f3fd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOfOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOfOperatorConversion.java @@ -48,7 +48,7 @@ public class ArrayOrdinalOfOperatorConversion implements SqlOperatorConversion ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.INTEGER) + .returnTypeNonNull(SqlTypeName.INTEGER) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOperatorConversion.java index 1dd9c4a800d0..c3d1302dc41e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOrdinalOperatorConversion.java @@ -48,7 +48,7 @@ public class ArrayOrdinalOperatorConversion implements SqlOperatorConversion ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayToStringOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayToStringOperatorConversion.java index 3d9331fa7100..802a95e5a369 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayToStringOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayToStringOperatorConversion.java @@ -48,7 +48,7 @@ public class ArrayToStringOperatorConversion implements SqlOperatorConversion ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java index 4e45c5a0fd97..d77c20b5d787 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/BTrimOperatorConversion.java @@ -37,7 +37,7 @@ public class BTrimOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("BTRIM") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .requiredOperands(1) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java index 9e4bf3c3d50e..f496e0ab9671 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/DateTruncOperatorConversion.java @@ -67,7 +67,7 @@ public class DateTruncOperatorConversion implements SqlOperatorConversion .operatorBuilder("DATE_TRUNC") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.TIMESTAMP) .requiredOperands(2) - .returnType(SqlTypeName.TIMESTAMP) + .returnTypeNonNull(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/IPv4AddressStringifyOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/IPv4AddressStringifyOperatorConversion.java index 6bd02b41221e..134723188d2e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/IPv4AddressStringifyOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/IPv4AddressStringifyOperatorConversion.java @@ -39,7 +39,7 @@ public class IPv4AddressStringifyOperatorConversion extends DirectOperatorConver OperandTypes.family(SqlTypeFamily.INTEGER), OperandTypes.family(SqlTypeFamily.STRING) )) - .nullableReturnType(SqlTypeName.CHAR) + .returnTypeNullable(SqlTypeName.CHAR) .functionCategory(SqlFunctionCategory.USER_DEFINED_FUNCTION) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java index 6a2d4f59b6ee..2d13b02fcbbd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LPadOperatorConversion.java @@ -37,7 +37,7 @@ public class LPadOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("LPAD") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .requiredOperands(2) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java index d3ac45da0fc1..70ec0c97e621 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LTrimOperatorConversion.java @@ -37,7 +37,7 @@ public class LTrimOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("LTRIM") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .requiredOperands(1) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LeftOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LeftOperatorConversion.java index 7363974eb79e..252343cddba1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LeftOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/LeftOperatorConversion.java @@ -39,7 +39,7 @@ public class LeftOperatorConversion implements SqlOperatorConversion .operatorBuilder("LEFT") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MillisToTimestampOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MillisToTimestampOperatorConversion.java index 227279f5a2f7..e8b8e748a640 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MillisToTimestampOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MillisToTimestampOperatorConversion.java @@ -39,7 +39,7 @@ public class MillisToTimestampOperatorConversion implements SqlOperatorConversio private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("MILLIS_TO_TIMESTAMP") .operandTypes(SqlTypeFamily.EXACT_NUMERIC) - .returnType(SqlTypeName.TIMESTAMP) + .returnTypeNonNull(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringAppendOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringAppendOperatorConversion.java index fb4745425ada..d57d0fcb1a4a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringAppendOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringAppendOperatorConversion.java @@ -47,7 +47,7 @@ public class MultiValueStringAppendOperatorConversion implements SqlOperatorConv ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringConcatOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringConcatOperatorConversion.java index 03e1c7c63bca..455715fe8f5c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringConcatOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringConcatOperatorConversion.java @@ -44,7 +44,7 @@ public class MultiValueStringConcatOperatorConversion implements SqlOperatorConv ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringPrependOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringPrependOperatorConversion.java index 4b7cc968df1f..6fa04ce176cd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringPrependOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringPrependOperatorConversion.java @@ -47,7 +47,7 @@ public class MultiValueStringPrependOperatorConversion implements SqlOperatorCon ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringSliceOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringSliceOperatorConversion.java index cba6ee66eec9..f278b3bf7ebb 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringSliceOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/MultiValueStringSliceOperatorConversion.java @@ -52,7 +52,7 @@ public class MultiValueStringSliceOperatorConversion implements SqlOperatorConve ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java index 29b0906e4174..9fd710fb1cb4 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ParseLongOperatorConversion.java @@ -38,7 +38,7 @@ public class ParseLongOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder(NAME) .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) - .returnType(SqlTypeName.BIGINT) + .returnTypeNonNull(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.STRING) .requiredOperands(1) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java index 4fa27af75037..5c1665c7083c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/QueryLookupOperatorConversion.java @@ -40,7 +40,7 @@ public class QueryLookupOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("LOOKUP") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java index 68874ef86ba1..47c8eadc2f85 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RPadOperatorConversion.java @@ -37,7 +37,7 @@ public class RPadOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("RPAD") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .requiredOperands(2) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java index 8361a8a5e8f6..6aa8f1b28a6f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RTrimOperatorConversion.java @@ -37,7 +37,7 @@ public class RTrimOperatorConversion implements SqlOperatorConversion private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("RTRIM") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .requiredOperands(1) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java index 18da1b31bff5..b6469718255e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpExtractOperatorConversion.java @@ -39,7 +39,8 @@ public class RegexpExtractOperatorConversion implements SqlOperatorConversion .operatorBuilder("REGEXP_EXTRACT") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) .requiredOperands(2) - .returnType(SqlTypeName.VARCHAR) + .literalOperands(1, 2) + .returnTypeNullable(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .build(); @@ -71,9 +72,13 @@ public DruidExpression toDruidExpression( : null; if (arg.isSimpleExtraction() && patternExpr.isLiteral() && (indexExpr == null || indexExpr.isLiteral())) { + final String pattern = (String) patternExpr.getLiteralValue(); + return arg.getSimpleExtraction().cascade( new RegexDimExtractionFn( - (String) patternExpr.getLiteralValue(), + // Undo the empty-to-null conversion from patternExpr parsing (patterns cannot be null, even in + // non-SQL-compliant null handling mode). + StringUtils.nullToEmptyNonDruidDataString(pattern), indexExpr == null ? DEFAULT_INDEX : ((Number) indexExpr.getLiteralValue()).intValue(), true, null diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpLikeOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpLikeOperatorConversion.java new file mode 100644 index 000000000000..ea699abe0215 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RegexpLikeOperatorConversion.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.sql.calcite.expression.builtin; + +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.rex.RexLiteral; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.sql.SqlFunction; +import org.apache.calcite.sql.SqlFunctionCategory; +import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.RegexDimFilter; +import org.apache.druid.segment.VirtualColumn; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.sql.calcite.expression.DruidExpression; +import org.apache.druid.sql.calcite.expression.Expressions; +import org.apache.druid.sql.calcite.expression.OperatorConversions; +import org.apache.druid.sql.calcite.expression.SqlOperatorConversion; +import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; + +import javax.annotation.Nullable; +import java.util.List; + +public class RegexpLikeOperatorConversion implements SqlOperatorConversion +{ + private static final SqlFunction SQL_FUNCTION = OperatorConversions + .operatorBuilder("REGEXP_LIKE") + .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) + .requiredOperands(2) + .literalOperands(1) + .returnTypeNonNull(SqlTypeName.BOOLEAN) + .functionCategory(SqlFunctionCategory.STRING) + .build(); + + @Override + public SqlFunction calciteOperator() + { + return SQL_FUNCTION; + } + + @Override + public DruidExpression toDruidExpression( + final PlannerContext plannerContext, + final RowSignature rowSignature, + final RexNode rexNode + ) + { + return OperatorConversions.convertCall( + plannerContext, + rowSignature, + rexNode, + operands -> DruidExpression.fromFunctionCall("regexp_like", operands) + ); + } + + @Nullable + @Override + public DimFilter toDruidFilter( + final PlannerContext plannerContext, + final RowSignature rowSignature, + @Nullable final VirtualColumnRegistry virtualColumnRegistry, + final RexNode rexNode + ) + { + final List operands = ((RexCall) rexNode).getOperands(); + final DruidExpression druidExpression = Expressions.toDruidExpression( + plannerContext, + rowSignature, + operands.get(0) + ); + + if (druidExpression == null) { + return null; + } + + final String pattern = RexLiteral.stringValue(operands.get(1)); + + if (druidExpression.isSimpleExtraction()) { + return new RegexDimFilter( + druidExpression.getSimpleExtraction().getColumn(), + pattern, + druidExpression.getSimpleExtraction().getExtractionFn(), + null + ); + } else if (virtualColumnRegistry != null) { + VirtualColumn v = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( + plannerContext, + druidExpression, + operands.get(0).getType().getSqlTypeName() + ); + + return new RegexDimFilter(v.getOutputName(), pattern, null, null); + } else { + return null; + } + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RepeatOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RepeatOperatorConversion.java index 318e95d9b4b0..9521a0443bcc 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RepeatOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RepeatOperatorConversion.java @@ -39,7 +39,7 @@ public class RepeatOperatorConversion implements SqlOperatorConversion .operatorBuilder("REPEAT") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReverseOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReverseOperatorConversion.java index 7cb0f0552976..70280abf2f98 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReverseOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ReverseOperatorConversion.java @@ -37,7 +37,7 @@ public class ReverseOperatorConversion implements SqlOperatorConversion .operatorBuilder("REVERSE") .operandTypes(SqlTypeFamily.CHARACTER) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RightOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RightOperatorConversion.java index 9fc3736d9f60..863bbccd5578 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RightOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/RightOperatorConversion.java @@ -39,7 +39,7 @@ public class RightOperatorConversion implements SqlOperatorConversion .operatorBuilder("RIGHT") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringFormatOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringFormatOperatorConversion.java index 3a535cdd9eab..b2aabbb2d11c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringFormatOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringFormatOperatorConversion.java @@ -42,7 +42,7 @@ public class StringFormatOperatorConversion implements SqlOperatorConversion .operatorBuilder("STRING_FORMAT") .operandTypeChecker(new StringFormatOperandTypeChecker()) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringToMultiValueStringOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringToMultiValueStringOperatorConversion.java index bdb590888aaa..7a6442f2dc8c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringToMultiValueStringOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StringToMultiValueStringOperatorConversion.java @@ -45,7 +45,7 @@ public class StringToMultiValueStringOperatorConversion implements SqlOperatorCo ) ) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StrposOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StrposOperatorConversion.java index 336c9d3b3723..e18c0896a5d4 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StrposOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/StrposOperatorConversion.java @@ -38,7 +38,7 @@ public class StrposOperatorConversion implements SqlOperatorConversion .operatorBuilder("STRPOS") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .functionCategory(SqlFunctionCategory.STRING) - .returnType(SqlTypeName.INTEGER) + .returnTypeNonNull(SqlTypeName.INTEGER) .build(); @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java index 09652ee08ff6..ee160d6b3ef6 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TextcatOperatorConversion.java @@ -36,7 +36,7 @@ public class TextcatOperatorConversion implements SqlOperatorConversion .operatorBuilder("textcat") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .requiredOperands(2) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.STRING) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java index 8b2ee0c7e78b..81b2dfa12ae2 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeCeilOperatorConversion.java @@ -41,7 +41,7 @@ public class TimeCeilOperatorConversion implements SqlOperatorConversion .operatorBuilder("TIME_CEIL") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER) .requiredOperands(2) - .returnType(SqlTypeName.TIMESTAMP) + .returnTypeNonNull(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java index ea041cfee2a9..35accd1f9b3f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeExtractOperatorConversion.java @@ -44,7 +44,7 @@ public class TimeExtractOperatorConversion implements SqlOperatorConversion .operatorBuilder("TIME_EXTRACT") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .requiredOperands(2) - .returnType(SqlTypeName.BIGINT) + .returnTypeNonNull(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java index af2716e39c63..87c07f25b7e5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFloorOperatorConversion.java @@ -56,7 +56,7 @@ public class TimeFloorOperatorConversion implements SqlOperatorConversion .operatorBuilder("TIME_FLOOR") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER) .requiredOperands(2) - .returnType(SqlTypeName.TIMESTAMP) + .returnTypeNonNull(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java index fa988a38a976..1f7b6f95d32e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeFormatOperatorConversion.java @@ -47,7 +47,7 @@ public class TimeFormatOperatorConversion implements SqlOperatorConversion .operatorBuilder("TIME_FORMAT") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .requiredOperands(1) - .returnType(SqlTypeName.VARCHAR) + .returnTypeNonNull(SqlTypeName.VARCHAR) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java index 8bba7321ac46..c79d981af2df 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeParseOperatorConversion.java @@ -45,7 +45,7 @@ public class TimeParseOperatorConversion implements SqlOperatorConversion .operatorBuilder("TIME_PARSE") .operandTypes(SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER, SqlTypeFamily.CHARACTER) .requiredOperands(1) - .nullableReturnType(SqlTypeName.TIMESTAMP) + .returnTypeNullable(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java index 4ca033a29141..25b05c40f1d1 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimeShiftOperatorConversion.java @@ -45,7 +45,7 @@ public class TimeShiftOperatorConversion implements SqlOperatorConversion .operatorBuilder("TIME_SHIFT") .operandTypes(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.CHARACTER, SqlTypeFamily.INTEGER, SqlTypeFamily.CHARACTER) .requiredOperands(3) - .returnType(SqlTypeName.TIMESTAMP) + .returnTypeNonNull(SqlTypeName.TIMESTAMP) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimestampToMillisOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimestampToMillisOperatorConversion.java index ec1742d03c8d..ae4565579fb7 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimestampToMillisOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/TimestampToMillisOperatorConversion.java @@ -39,7 +39,7 @@ public class TimestampToMillisOperatorConversion implements SqlOperatorConversio private static final SqlFunction SQL_FUNCTION = OperatorConversions .operatorBuilder("TIMESTAMP_TO_MILLIS") .operandTypes(SqlTypeFamily.TIMESTAMP) - .returnType(SqlTypeName.BIGINT) + .returnTypeNonNull(SqlTypeName.BIGINT) .functionCategory(SqlFunctionCategory.TIMEDATE) .build(); 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 0fd811a0d28b..3f7699fa9f18 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 @@ -83,6 +83,7 @@ import org.apache.druid.sql.calcite.expression.builtin.RPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RTrimOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.RegexpLikeOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReinterpretOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RepeatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion; @@ -162,6 +163,7 @@ public class DruidOperatorTable implements SqlOperatorTable .add(new LTrimOperatorConversion()) .add(new PositionOperatorConversion()) .add(new RegexpExtractOperatorConversion()) + .add(new RegexpLikeOperatorConversion()) .add(new RTrimOperatorConversion()) .add(new ParseLongOperatorConversion()) .add(new StringFormatOperatorConversion()) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java index 31244d13b497..c181cb8c885a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/DruidRexExecutor.java @@ -121,10 +121,13 @@ public void reduce( } else { if (exprResult.type() == ExprType.LONG) { bigDecimal = BigDecimal.valueOf(exprResult.asLong()); + } else { + // if exprResult evaluates to Nan or infinity, this will throw a NumberFormatException. + // If you find yourself in such a position, consider casting the literal to a BIGINT so that + // the query can execute. bigDecimal = BigDecimal.valueOf(exprResult.asDouble()); } - literal = rexBuilder.makeLiteral(bigDecimal, constExp.getType(), true); } } else if (sqlTypeName == SqlTypeName.ARRAY) { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java index acc27c71b8df..03b1a31c3689 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Rules.java @@ -123,14 +123,17 @@ public class Rules ProjectTableScanRule.INTERPRETER ); - // Rules from RelOptUtil's registerReductionRules. + // Rules from RelOptUtil's registerReductionRules, minus: + // + // 1) ReduceExpressionsRule.JOIN_INSTANCE + // Removed by https://github.com/apache/druid/pull/9941 due to issue in https://github.com/apache/druid/issues/9942 + // TODO: Re-enable when https://github.com/apache/druid/issues/9942 is fixed private static final List REDUCTION_RULES = ImmutableList.of( ReduceExpressionsRule.PROJECT_INSTANCE, ReduceExpressionsRule.FILTER_INSTANCE, ReduceExpressionsRule.CALC_INSTANCE, ReduceExpressionsRule.WINDOW_INSTANCE, - ReduceExpressionsRule.JOIN_INSTANCE, ValuesReduceRule.FILTER_INSTANCE, ValuesReduceRule.PROJECT_FILTER_INSTANCE, ValuesReduceRule.PROJECT_INSTANCE, @@ -166,6 +169,9 @@ public class Rules // 2) SemiJoinRule.PROJECT and SemiJoinRule.JOIN (we don't need to detect semi-joins, because they are handled // fine as-is by DruidJoinRule). // 3) JoinCommuteRule (we don't support reordering joins yet). + // 4) FilterJoinRule.FILTER_ON_JOIN and FilterJoinRule.JOIN + // Removed by https://github.com/apache/druid/pull/9773 due to issue in https://github.com/apache/druid/issues/9843 + // TODO: Re-enable when https://github.com/apache/druid/issues/9843 is fixed private static final List ABSTRACT_RELATIONAL_RULES = ImmutableList.of( AbstractConverter.ExpandConversionRule.INSTANCE, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java index 1655d73e5445..6faf2198627a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidJoinQueryRel.java @@ -337,6 +337,8 @@ private static boolean computeLeftRequiresSubquery(final DruidRel left) private static boolean computeRightRequiresSubquery(final DruidRel right) { // Right requires a subquery unless it's a scan or mapping on top of a global datasource. + // ideally this would involve JoinableFactory.isDirectlyJoinable to check that the global datasources + // are in fact possibly joinable, but for now isGlobal is coupled to joinability return !(DruidRels.isScanOrMapping(right, false) && DruidRels.dataSourceIfLeafRel(right).filter(DataSource::isGlobal).isPresent()); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java index f63c618253bc..32e4b746d7ee 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidOuterQueryRel.java @@ -36,7 +36,6 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.QueryDataSource; import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.table.RowSignatures; @@ -117,17 +116,10 @@ public DruidOuterQueryRel withPartialQuery(final PartialDruidQuery newQueryBuild public DruidQuery toDruidQuery(final boolean finalizeAggregations) { // Must finalize aggregations on subqueries. - final DruidQuery subQuery = ((DruidRel) sourceRel).toDruidQuery(true); - - final GroupByQuery groupByQuery = subQuery.toGroupByQuery(); - if (groupByQuery == null) { - throw new CannotBuildQueryException("Subquery could not be converted to GroupBy query"); - } - final RowSignature sourceRowSignature = subQuery.getOutputRowSignature(); return partialQuery.build( - new QueryDataSource(groupByQuery), + new QueryDataSource(subQuery.getQuery()), sourceRowSignature, getPlannerContext(), getCluster().getRexBuilder(), 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 262ba664def1..137f9b70bd38 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 @@ -191,7 +191,7 @@ public static DruidQuery fromPartialQuery( computeGrouping( partialQuery, plannerContext, - computeOutputRowSignature(sourceRowSignature, selectProjection, null, null), + computeOutputRowSignature(sourceRowSignature, null, null, null), virtualColumnRegistry, rexBuilder, finalizeAggregations @@ -433,7 +433,13 @@ private static Subtotals computeSubtotals( final Aggregate aggregate = partialQuery.getAggregate(); // dimBitMapping maps from input field position to group set position (dimension number). - final int[] dimBitMapping = new int[rowSignature.size()]; + final int[] dimBitMapping; + if (partialQuery.getSelectProject() != null) { + dimBitMapping = new int[partialQuery.getSelectProject().getRowType().getFieldCount()]; + } else { + dimBitMapping = new int[rowSignature.size()]; + } + int i = 0; for (int dimBit : aggregate.getGroupSet()) { dimBitMapping[dimBit] = i++; @@ -734,6 +740,9 @@ public TimeseriesQuery toTimeseriesQuery() final Granularity queryGranularity; final boolean descending; int timeseriesLimit = 0; + final Map theContext = new HashMap<>(); + theContext.put("skipEmptyBuckets", true); + theContext.putAll(plannerContext.getQueryContext()); if (grouping.getDimensions().isEmpty()) { queryGranularity = Granularities.ALL; descending = false; @@ -748,7 +757,10 @@ public TimeseriesQuery toTimeseriesQuery() // Timeseries only applies if the single dimension is granular __time. return null; } - + theContext.put( + TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, + Iterables.getOnlyElement(grouping.getDimensions()).toDimensionSpec().getOutputName() + ); if (sorting != null) { // If there is sorting, set timeseriesLimit to given value if less than Integer.Max_VALUE if (sorting.isLimited()) { @@ -782,9 +794,6 @@ public TimeseriesQuery toTimeseriesQuery() if (sorting != null && sorting.getProjection() != null) { postAggregators.addAll(sorting.getProjection().getPostAggregators()); } - final Map theContext = new HashMap<>(); - theContext.put("skipEmptyBuckets", true); - theContext.putAll(plannerContext.getQueryContext()); return new TimeseriesQuery( dataSource, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java index d73312e98ac8..f8c3dc2a152d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/QueryMaker.java @@ -100,9 +100,8 @@ public Sequence runQuery(final DruidQuery druidQuery) if (query instanceof TimeseriesQuery && !druidQuery.getGrouping().getDimensions().isEmpty()) { // Hack for timeseries queries: when generating them, DruidQuery.toTimeseriesQuery translates a dimension // based on a timestamp_floor expression into a 'granularity'. This is not reflected in the druidQuery's - // output row signature, so we have to account for it here. When groupBy on timestamp_floor expressions is - // just as fast as a timeseries query (a noble goal) we can remove timeseries queries from the SQL layer and - // also remove this hack. + // output row signature, so we have to account for it here. + // TODO: We can remove this once https://github.com/apache/druid/issues/9974 is done. final String timeDimension = Iterables.getOnlyElement(druidQuery.getGrouping().getDimensions()).getOutputName(); rowOrder = druidQuery.getOutputRowSignature().getColumnNames().stream() .map(f -> timeDimension.equals(f) ? ColumnHolder.TIME_COLUMN_NAME : f) @@ -127,7 +126,7 @@ private List findBaseDataSourceIntervals(Query query) return DataSourceAnalysis.forDataSource(query.getDataSource()) .getBaseQuerySegmentSpec() .map(QuerySegmentSpec::getIntervals) - .orElse(query.getIntervals()); + .orElseGet(query::getIntervals); } private Sequence execute(Query query, final List newFields, final List newTypes) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java index 9f087eaf9ccd..b264749308c6 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/DruidSchema.java @@ -47,6 +47,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.metadata.metadata.AllColumnIncluderator; import org.apache.druid.query.metadata.metadata.ColumnAnalysis; @@ -55,7 +56,9 @@ import org.apache.druid.query.spec.MultipleSpecificSegmentSpec; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.join.JoinableFactory; import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.security.AuthenticationResult; @@ -100,7 +103,9 @@ public class DruidSchema extends AbstractSchema private final QueryLifecycleFactory queryLifecycleFactory; private final PlannerConfig config; + private final SegmentManager segmentManager; private final ViewManager viewManager; + private final JoinableFactory joinableFactory; private final ExecutorService cacheExec; private final ConcurrentMap tables; @@ -117,26 +122,35 @@ public class DruidSchema extends AbstractSchema private int totalSegments = 0; // All mutable segments. + @GuardedBy("lock") private final Set mutableSegments = new TreeSet<>(SEGMENT_ORDER); // All dataSources that need tables regenerated. + @GuardedBy("lock") private final Set dataSourcesNeedingRebuild = new HashSet<>(); // All segments that need to be refreshed. + @GuardedBy("lock") private final TreeSet segmentsNeedingRefresh = new TreeSet<>(SEGMENT_ORDER); // Escalator, so we can attach an authentication result to queries we generate. private final Escalator escalator; + @GuardedBy("lock") private boolean refreshImmediately = false; + @GuardedBy("lock") private long lastRefresh = 0L; + @GuardedBy("lock") private long lastFailure = 0L; + @GuardedBy("lock") private boolean isServerViewInitialized = false; @Inject public DruidSchema( final QueryLifecycleFactory queryLifecycleFactory, final TimelineServerView serverView, + final SegmentManager segmentManager, + final JoinableFactory joinableFactory, final PlannerConfig config, final ViewManager viewManager, final Escalator escalator @@ -144,6 +158,8 @@ public DruidSchema( { this.queryLifecycleFactory = Preconditions.checkNotNull(queryLifecycleFactory, "queryLifecycleFactory"); Preconditions.checkNotNull(serverView, "serverView"); + this.segmentManager = segmentManager; + this.joinableFactory = joinableFactory; this.config = Preconditions.checkNotNull(config, "config"); this.viewManager = Preconditions.checkNotNull(viewManager, "viewManager"); this.cacheExec = Execs.singleThreaded("DruidSchema-Cache-%d"); @@ -196,118 +212,114 @@ public ServerView.CallbackAction serverSegmentRemoved( public void start() throws InterruptedException { cacheExec.submit( - new Runnable() - { - @Override - public void run() - { - try { - while (!Thread.currentThread().isInterrupted()) { - final Set segmentsToRefresh = new TreeSet<>(); - final Set dataSourcesToRebuild = new TreeSet<>(); - - try { - synchronized (lock) { - final long nextRefreshNoFuzz = DateTimes - .utc(lastRefresh) - .plus(config.getMetadataRefreshPeriod()) - .getMillis(); - - // Fuzz a bit to spread load out when we have multiple brokers. - final long nextRefresh = nextRefreshNoFuzz + (long) ((nextRefreshNoFuzz - lastRefresh) * 0.10); - - while (true) { - // Do not refresh if it's too soon after a failure (to avoid rapid cycles of failure). - final boolean wasRecentFailure = DateTimes.utc(lastFailure) - .plus(config.getMetadataRefreshPeriod()) - .isAfterNow(); - - if (isServerViewInitialized && - !wasRecentFailure && - (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()) && - (refreshImmediately || nextRefresh < System.currentTimeMillis())) { - // We need to do a refresh. Break out of the waiting loop. - break; - } - - if (isServerViewInitialized) { - // Server view is initialized, but we don't need to do a refresh. Could happen if there are - // no segments in the system yet. Just mark us as initialized, then. - initialized.countDown(); - } - - // Wait some more, we'll wake up when it might be time to do another refresh. - lock.wait(Math.max(1, nextRefresh - System.currentTimeMillis())); + () -> { + try { + while (!Thread.currentThread().isInterrupted()) { + final Set segmentsToRefresh = new TreeSet<>(); + final Set dataSourcesToRebuild = new TreeSet<>(); + + try { + synchronized (lock) { + final long nextRefreshNoFuzz = DateTimes + .utc(lastRefresh) + .plus(config.getMetadataRefreshPeriod()) + .getMillis(); + + // Fuzz a bit to spread load out when we have multiple brokers. + final long nextRefresh = nextRefreshNoFuzz + (long) ((nextRefreshNoFuzz - lastRefresh) * 0.10); + + while (true) { + // Do not refresh if it's too soon after a failure (to avoid rapid cycles of failure). + final boolean wasRecentFailure = DateTimes.utc(lastFailure) + .plus(config.getMetadataRefreshPeriod()) + .isAfterNow(); + + if (isServerViewInitialized && + !wasRecentFailure && + (!segmentsNeedingRefresh.isEmpty() || !dataSourcesNeedingRebuild.isEmpty()) && + (refreshImmediately || nextRefresh < System.currentTimeMillis())) { + // We need to do a refresh. Break out of the waiting loop. + break; } - segmentsToRefresh.addAll(segmentsNeedingRefresh); - segmentsNeedingRefresh.clear(); - - // Mutable segments need a refresh every period, since new columns could be added dynamically. - segmentsNeedingRefresh.addAll(mutableSegments); + if (isServerViewInitialized) { + // Server view is initialized, but we don't need to do a refresh. Could happen if there are + // no segments in the system yet. Just mark us as initialized, then. + initialized.countDown(); + } - lastFailure = 0L; - lastRefresh = System.currentTimeMillis(); - refreshImmediately = false; + // Wait some more, we'll wake up when it might be time to do another refresh. + lock.wait(Math.max(1, nextRefresh - System.currentTimeMillis())); } - // Refresh the segments. - final Set refreshed = refreshSegments(segmentsToRefresh); + segmentsToRefresh.addAll(segmentsNeedingRefresh); + segmentsNeedingRefresh.clear(); - synchronized (lock) { - // Add missing segments back to the refresh list. - segmentsNeedingRefresh.addAll(Sets.difference(segmentsToRefresh, refreshed)); + // Mutable segments need a refresh every period, since new columns could be added dynamically. + segmentsNeedingRefresh.addAll(mutableSegments); - // Compute the list of dataSources to rebuild tables for. - dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild); - refreshed.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); - dataSourcesNeedingRebuild.clear(); + lastFailure = 0L; + lastRefresh = System.currentTimeMillis(); + refreshImmediately = false; + } - lock.notifyAll(); - } + // Refresh the segments. + final Set refreshed = refreshSegments(segmentsToRefresh); - // Rebuild the dataSources. - for (String dataSource : dataSourcesToRebuild) { - final DruidTable druidTable = buildDruidTable(dataSource); - final DruidTable oldTable = tables.put(dataSource, druidTable); - if (oldTable == null || !oldTable.getRowSignature().equals(druidTable.getRowSignature())) { - log.info("dataSource [%s] has new signature: %s.", dataSource, druidTable.getRowSignature()); - } else { - log.debug("dataSource [%s] signature is unchanged.", dataSource); - } - } + synchronized (lock) { + // Add missing segments back to the refresh list. + segmentsNeedingRefresh.addAll(Sets.difference(segmentsToRefresh, refreshed)); - initialized.countDown(); - } - catch (InterruptedException e) { - // Fall through. - throw e; + // Compute the list of dataSources to rebuild tables for. + dataSourcesToRebuild.addAll(dataSourcesNeedingRebuild); + refreshed.forEach(segment -> dataSourcesToRebuild.add(segment.getDataSource())); + dataSourcesNeedingRebuild.clear(); + + lock.notifyAll(); } - catch (Exception e) { - log.warn(e, "Metadata refresh failed, trying again soon."); - - synchronized (lock) { - // Add our segments and dataSources back to their refresh and rebuild lists. - segmentsNeedingRefresh.addAll(segmentsToRefresh); - dataSourcesNeedingRebuild.addAll(dataSourcesToRebuild); - lastFailure = System.currentTimeMillis(); - lock.notifyAll(); + + // Rebuild the dataSources. + for (String dataSource : dataSourcesToRebuild) { + final DruidTable druidTable = buildDruidTable(dataSource); + final DruidTable oldTable = tables.put(dataSource, druidTable); + final String description = druidTable.getDataSource().isGlobal() ? "global dataSource" : "dataSource"; + if (oldTable == null || !oldTable.getRowSignature().equals(druidTable.getRowSignature())) { + log.info("%s [%s] has new signature: %s.", description, dataSource, druidTable.getRowSignature()); + } else { + log.debug("%s [%s] signature is unchanged.", description, dataSource); } } + + initialized.countDown(); + } + catch (InterruptedException e) { + // Fall through. + throw e; + } + catch (Exception e) { + log.warn(e, "Metadata refresh failed, trying again soon."); + + synchronized (lock) { + // Add our segments and dataSources back to their refresh and rebuild lists. + segmentsNeedingRefresh.addAll(segmentsToRefresh); + dataSourcesNeedingRebuild.addAll(dataSourcesToRebuild); + lastFailure = System.currentTimeMillis(); + lock.notifyAll(); + } } } - catch (InterruptedException e) { - // Just exit. - } - catch (Throwable e) { - // Throwables that fall out to here (not caught by an inner try/catch) are potentially gnarly, like - // OOMEs. Anyway, let's just emit an alert and stop refreshing metadata. - log.makeAlert(e, "Metadata refresh failed permanently").emit(); - throw e; - } - finally { - log.info("Metadata refresh stopped."); - } + } + catch (InterruptedException e) { + // Just exit. + } + catch (Throwable e) { + // Throwables that fall out to here (not caught by an inner try/catch) are potentially gnarly, like + // OOMEs. Anyway, let's just emit an alert and stop refreshing metadata. + log.makeAlert(e, "Metadata refresh failed permanently").emit(); + throw e; + } + finally { + log.info("Metadata refresh stopped."); } } ); @@ -351,44 +363,50 @@ protected Multimap getFunctionMultim void addSegment(final DruidServerMetadata server, final DataSegment segment) { synchronized (lock) { - final Map knownSegments = segmentMetadataInfo.get(segment.getDataSource()); - AvailableSegmentMetadata segmentMetadata = knownSegments != null ? knownSegments.get(segment.getId()) : null; - if (segmentMetadata == null) { - // segmentReplicatable is used to determine if segments are served by historical or realtime servers - long isRealtime = server.segmentReplicatable() ? 0 : 1; - segmentMetadata = AvailableSegmentMetadata.builder( - segment, - isRealtime, - ImmutableSet.of(server), - null, - DEFAULT_NUM_ROWS - ).build(); - // Unknown segment. - setAvailableSegmentMetadata(segment.getId(), segmentMetadata); - segmentsNeedingRefresh.add(segment.getId()); - if (!server.segmentReplicatable()) { - log.debug("Added new mutable segment[%s].", segment.getId()); - mutableSegments.add(segment.getId()); - } else { - log.debug("Added new immutable segment[%s].", segment.getId()); - } + if (server.getType().equals(ServerType.BROKER)) { + // a segment on a broker means a broadcast datasource, skip metadata because we'll also see this segment on the + // historical, however mark the datasource for refresh because it needs to be globalized + dataSourcesNeedingRebuild.add(segment.getDataSource()); } else { - final Set segmentServers = segmentMetadata.getReplicas(); - final ImmutableSet servers = new ImmutableSet.Builder() - .addAll(segmentServers) - .add(server) - .build(); - final AvailableSegmentMetadata metadataWithNumReplicas = AvailableSegmentMetadata - .from(segmentMetadata) - .withReplicas(servers) - .withRealtime(recomputeIsRealtime(servers)) - .build(); - knownSegments.put(segment.getId(), metadataWithNumReplicas); - if (server.segmentReplicatable()) { - // If a segment shows up on a replicatable (historical) server at any point, then it must be immutable, - // even if it's also available on non-replicatable (realtime) servers. - mutableSegments.remove(segment.getId()); - log.debug("Segment[%s] has become immutable.", segment.getId()); + final Map knownSegments = segmentMetadataInfo.get(segment.getDataSource()); + AvailableSegmentMetadata segmentMetadata = knownSegments != null ? knownSegments.get(segment.getId()) : null; + if (segmentMetadata == null) { + // segmentReplicatable is used to determine if segments are served by historical or realtime servers + long isRealtime = server.isSegmentReplicationTarget() ? 0 : 1; + segmentMetadata = AvailableSegmentMetadata.builder( + segment, + isRealtime, + ImmutableSet.of(server), + null, + DEFAULT_NUM_ROWS + ).build(); + // Unknown segment. + setAvailableSegmentMetadata(segment.getId(), segmentMetadata); + segmentsNeedingRefresh.add(segment.getId()); + if (!server.isSegmentReplicationTarget()) { + log.debug("Added new mutable segment[%s].", segment.getId()); + mutableSegments.add(segment.getId()); + } else { + log.debug("Added new immutable segment[%s].", segment.getId()); + } + } else { + final Set segmentServers = segmentMetadata.getReplicas(); + final ImmutableSet servers = new ImmutableSet.Builder() + .addAll(segmentServers) + .add(server) + .build(); + final AvailableSegmentMetadata metadataWithNumReplicas = AvailableSegmentMetadata + .from(segmentMetadata) + .withReplicas(servers) + .withRealtime(recomputeIsRealtime(servers)) + .build(); + knownSegments.put(segment.getId(), metadataWithNumReplicas); + if (server.isSegmentReplicationTarget()) { + // If a segment shows up on a replicatable (historical) server at any point, then it must be immutable, + // even if it's also available on non-replicatable (realtime) servers. + mutableSegments.remove(segment.getId()); + log.debug("Segment[%s] has become immutable.", segment.getId()); + } } } if (!tables.containsKey(segment.getDataSource())) { @@ -430,20 +448,26 @@ void removeServerSegment(final DruidServerMetadata server, final DataSegment seg { synchronized (lock) { log.debug("Segment[%s] is gone from server[%s]", segment.getId(), server.getName()); - final Map knownSegments = segmentMetadataInfo.get(segment.getDataSource()); - final AvailableSegmentMetadata segmentMetadata = knownSegments.get(segment.getId()); - final Set segmentServers = segmentMetadata.getReplicas(); - final ImmutableSet servers = FluentIterable - .from(segmentServers) - .filter(Predicates.not(Predicates.equalTo(server))) - .toSet(); - - final AvailableSegmentMetadata metadataWithNumReplicas = AvailableSegmentMetadata - .from(segmentMetadata) - .withReplicas(servers) - .withRealtime(recomputeIsRealtime(servers)) - .build(); - knownSegments.put(segment.getId(), metadataWithNumReplicas); + if (server.getType().equals(ServerType.BROKER)) { + // a segment on a broker means a broadcast datasource, skip metadata because we'll also see this segment on the + // historical, however mark the datasource for refresh because it might no longer be broadcast or something + dataSourcesNeedingRebuild.add(segment.getDataSource()); + } else { + final Map knownSegments = segmentMetadataInfo.get(segment.getDataSource()); + final AvailableSegmentMetadata segmentMetadata = knownSegments.get(segment.getId()); + final Set segmentServers = segmentMetadata.getReplicas(); + final ImmutableSet servers = FluentIterable + .from(segmentServers) + .filter(Predicates.not(Predicates.equalTo(server))) + .toSet(); + + final AvailableSegmentMetadata metadataWithNumReplicas = AvailableSegmentMetadata + .from(segmentMetadata) + .withReplicas(servers) + .withRealtime(recomputeIsRealtime(servers)) + .build(); + knownSegments.put(segment.getId(), metadataWithNumReplicas); + } lock.notifyAll(); } } @@ -582,7 +606,7 @@ void setAvailableSegmentMetadata(final SegmentId segmentId, final AvailableSegme } } - private DruidTable buildDruidTable(final String dataSource) + protected DruidTable buildDruidTable(final String dataSource) { synchronized (lock) { final Map segmentMap = segmentMetadataInfo.get(dataSource); @@ -606,7 +630,23 @@ private DruidTable buildDruidTable(final String dataSource) final RowSignature.Builder builder = RowSignature.builder(); columnTypes.forEach(builder::add); - return new DruidTable(new TableDataSource(dataSource), builder.build()); + + final TableDataSource tableDataSource; + + // to be a GlobalTableDataSource instead of a TableDataSource, it must appear on all servers (inferred by existing + // in the segment cache, which in this case belongs to the broker meaning only broadcast segments live here) + // to be joinable, it must be possibly joinable according to the factory. we only consider broadcast datasources + // at this time, and isGlobal is currently strongly coupled with joinable, so only make a global table datasource + // if also joinable + final GlobalTableDataSource maybeGlobal = new GlobalTableDataSource(dataSource); + final boolean isJoinable = joinableFactory.isDirectlyJoinable(maybeGlobal); + final boolean isBroadcast = segmentManager.getDataSourceNames().contains(dataSource); + if (isBroadcast && isJoinable) { + tableDataSource = maybeGlobal; + } else { + tableDataSource = new TableDataSource(dataSource); + } + return new DruidTable(tableDataSource, builder.build(), isJoinable, isBroadcast); } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java index bf84ea131c8c..8ee93a21e631 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/InformationSchema.java @@ -53,6 +53,7 @@ import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceAction; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.table.DruidTable; import org.apache.druid.sql.calcite.table.RowSignatures; import javax.annotation.Nullable; @@ -83,6 +84,8 @@ public class InformationSchema extends AbstractSchema .add("TABLE_SCHEMA", ValueType.STRING) .add("TABLE_NAME", ValueType.STRING) .add("TABLE_TYPE", ValueType.STRING) + .add("IS_JOINABLE", ValueType.STRING) + .add("IS_BROADCAST", ValueType.STRING) .build(); private static final RowSignature COLUMNS_SIGNATURE = RowSignature .builder() @@ -109,6 +112,9 @@ public class InformationSchema extends AbstractSchema return Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName)); }; + private static final String INFO_TRUE = "YES"; + private static final String INFO_FALSE = "NO"; + private final SchemaPlus rootSchema; private final Map tableMap; private final AuthorizerMapper authorizerMapper; @@ -217,18 +223,27 @@ public Iterable apply(final String schemaName) return Iterables.filter( Iterables.concat( FluentIterable.from(authorizedTableNames).transform( - new Function() - { - @Override - public Object[] apply(final String tableName) - { - return new Object[]{ - CATALOG_NAME, // TABLE_CATALOG - schemaName, // TABLE_SCHEMA - tableName, // TABLE_NAME - subSchema.getTable(tableName).getJdbcTableType().toString() // TABLE_TYPE - }; + tableName -> { + final Table table = subSchema.getTable(tableName); + final boolean isJoinable; + final boolean isBroadcast; + if (table instanceof DruidTable) { + DruidTable druidTable = (DruidTable) table; + isJoinable = druidTable.isJoinable(); + isBroadcast = druidTable.isBroadcast(); + } else { + isJoinable = false; + isBroadcast = false; } + + return new Object[]{ + CATALOG_NAME, // TABLE_CATALOG + schemaName, // TABLE_SCHEMA + tableName, // TABLE_NAME + table.getJdbcTableType().toString(), // TABLE_TYPE + isJoinable ? INFO_TRUE : INFO_FALSE, // IS_JOINABLE + isBroadcast ? INFO_TRUE : INFO_FALSE // IS_BROADCAST + }; } ), FluentIterable.from(authorizedFunctionNames).transform( @@ -242,7 +257,9 @@ public Object[] apply(final String functionName) CATALOG_NAME, // TABLE_CATALOG schemaName, // TABLE_SCHEMA functionName, // TABLE_NAME - "VIEW" // TABLE_TYPE + "VIEW", // TABLE_TYPE + INFO_FALSE, // IS_JOINABLE + INFO_FALSE // IS_BROADCAST }; } else { return null; @@ -406,7 +423,7 @@ public Object[] apply(final RelDataTypeField field) field.getName(), // COLUMN_NAME String.valueOf(field.getIndex()), // ORDINAL_POSITION "", // COLUMN_DEFAULT - type.isNullable() ? "YES" : "NO", // IS_NULLABLE + type.isNullable() ? INFO_TRUE : INFO_FALSE, // IS_NULLABLE type.getSqlTypeName().toString(), // DATA_TYPE null, // CHARACTER_MAXIMUM_LENGTH null, // CHARACTER_OCTET_LENGTH diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java index b3f331471791..6ddeaabb445d 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/LookupSchema.java @@ -57,7 +57,9 @@ protected Map getTableMap() final ImmutableMap.Builder tableMapBuilder = ImmutableMap.builder(); for (final String lookupName : lookupProvider.getAllLookupNames()) { - tableMapBuilder.put(lookupName, new DruidTable(new LookupDataSource(lookupName), ROW_SIGNATURE)); + // all lookups should be also joinable through lookup joinable factory, and lookups are effectively broadcast + // (if we ignore lookup tiers...) + tableMapBuilder.put(lookupName, new DruidTable(new LookupDataSource(lookupName), ROW_SIGNATURE, true, true)); } return tableMapBuilder.build(); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/table/DruidTable.java b/sql/src/main/java/org/apache/druid/sql/calcite/table/DruidTable.java index 521a051b7c98..94da5ede16e5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/table/DruidTable.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/table/DruidTable.java @@ -41,14 +41,20 @@ public class DruidTable implements TranslatableTable { private final DataSource dataSource; private final RowSignature rowSignature; + private final boolean joinable; + private final boolean broadcast; public DruidTable( final DataSource dataSource, - final RowSignature rowSignature + final RowSignature rowSignature, + final boolean isJoinable, + final boolean isBroadcast ) { this.dataSource = Preconditions.checkNotNull(dataSource, "dataSource"); this.rowSignature = Preconditions.checkNotNull(rowSignature, "rowSignature"); + this.joinable = isJoinable; + this.broadcast = isBroadcast; } public DataSource getDataSource() @@ -61,6 +67,16 @@ public RowSignature getRowSignature() return rowSignature; } + public boolean isJoinable() + { + return joinable; + } + + public boolean isBroadcast() + { + return broadcast; + } + @Override public Schema.TableType getJdbcTableType() { diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index c09aa858da6a..1c950c94819c 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.server.QueryCapacityExceededException; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.SqlLifecycle; @@ -176,6 +177,11 @@ public Response doPost( lifecycle.emitLogsAndMetrics(cap, remoteAddr, -1); return Response.status(QueryCapacityExceededException.STATUS_CODE).entity(jsonMapper.writeValueAsBytes(cap)).build(); } + catch (QueryUnsupportedException unsupported) { + log.warn(unsupported, "Failed to handle query: %s", sqlQuery); + lifecycle.emitLogsAndMetrics(unsupported, remoteAddr, -1); + return Response.status(QueryUnsupportedException.STATUS_CODE).entity(jsonMapper.writeValueAsBytes(unsupported)).build(); + } catch (ForbiddenException e) { throw e; // let ForbiddenExceptionMapper handle this } diff --git a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java index 97c0f7c4bfb6..3f43f76c1757 100644 --- a/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java +++ b/sql/src/test/java/org/apache/druid/sql/avatica/DruidAvaticaHandlerTest.java @@ -384,6 +384,12 @@ public void testDatabaseMetaDataTables() throws Exception final DatabaseMetaData metaData = client.getMetaData(); Assert.assertEquals( ImmutableList.of( + row( + Pair.of("TABLE_CAT", "druid"), + Pair.of("TABLE_NAME", CalciteTests.BROADCAST_DATASOURCE), + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_TYPE", "TABLE") + ), row( Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1), @@ -414,8 +420,19 @@ public void testDatabaseMetaDataTables() throws Exception Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3), Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_TYPE", "TABLE") + ), + row( + Pair.of("TABLE_CAT", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_TYPE", "TABLE") + ), + row( + Pair.of("TABLE_CAT", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_TYPE", "TABLE") ) - ), getRows( metaData.getTables(null, "druid", "%", null), @@ -430,6 +447,12 @@ public void testDatabaseMetaDataTablesAsSuperuser() throws Exception final DatabaseMetaData metaData = superuserClient.getMetaData(); Assert.assertEquals( ImmutableList.of( + row( + Pair.of("TABLE_CAT", "druid"), + Pair.of("TABLE_NAME", CalciteTests.BROADCAST_DATASOURCE), + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_TYPE", "TABLE") + ), row( Pair.of("TABLE_CAT", "druid"), Pair.of("TABLE_NAME", CalciteTests.DATASOURCE1), @@ -465,8 +488,19 @@ public void testDatabaseMetaDataTablesAsSuperuser() throws Exception Pair.of("TABLE_NAME", CalciteTests.DATASOURCE3), Pair.of("TABLE_SCHEM", "druid"), Pair.of("TABLE_TYPE", "TABLE") + ), + row( + Pair.of("TABLE_CAT", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_TYPE", "TABLE") + ), + row( + Pair.of("TABLE_CAT", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_TYPE", "TABLE") ) - ), getRows( metaData.getTables(null, "druid", "%", null), @@ -957,6 +991,210 @@ public void testSysTableParameterBinding() throws Exception ); } + @Test + public void testEscapingForGetColumns() throws Exception + { + final DatabaseMetaData metaData = client.getMetaData(); + + ImmutableList> someDatasourceColumns = ImmutableList.of( + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "__time") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "cnt") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "dim1") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "dim2") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "dim3") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "m1") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "m2") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "unique_dim1") + ) + ); + // If the escape clause wasn't correctly set, rows for potentially none or more than + // one datasource (some_datasource and somexdatasource) would have been returned + Assert.assertEquals( + someDatasourceColumns, + getRows( + metaData.getColumns(null, "dr_id", CalciteTests.SOME_DATSOURCE_ESCAPED, null), + ImmutableSet.of("TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME") + ) + ); + + ImmutableList> someXDatasourceColumns = ImmutableList.of( + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "__time") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "cnt_x") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "m1_x") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "m2_x") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "unique_dim1_x") + ) + ); + Assert.assertEquals( + someXDatasourceColumns, + getRows( + metaData.getColumns(null, "dr_id", "somexdatasource", null), + ImmutableSet.of("TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME") + ) + ); + + List> columnsOfBothTables = new ArrayList<>(someDatasourceColumns); + columnsOfBothTables.addAll(someXDatasourceColumns); + // Assert that the pattern matching still works when no escape string is provided + Assert.assertEquals( + columnsOfBothTables, + getRows( + metaData.getColumns(null, "dr_id", "some_datasource", null), + ImmutableSet.of("TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME") + ) + ); + + // Assert column name pattern works correctly when _ is in the column names + Assert.assertEquals( + ImmutableList.of( + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "m1_x") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "m2_x") + ) + ), + getRows( + metaData.getColumns("druid", "dr_id", CalciteTests.SOMEXDATASOURCE, "m_\\_x"), + ImmutableSet.of("TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME") + ) + ); + + // Assert column name pattern with % works correctly for column names starting with m + Assert.assertEquals( + ImmutableList.of( + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "m1") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE), + Pair.of("COLUMN_NAME", "m2") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "m1_x") + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE), + Pair.of("COLUMN_NAME", "m2_x") + ) + ), + getRows( + metaData.getColumns("druid", "dr_id", CalciteTests.SOME_DATASOURCE, "m%"), + ImmutableSet.of("TABLE_NAME", "TABLE_SCHEM", "COLUMN_NAME") + ) + ); + } + + @Test + public void testEscapingForGetTables() throws Exception + { + final DatabaseMetaData metaData = client.getMetaData(); + + Assert.assertEquals( + ImmutableList.of( + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE) + ) + ), + getRows( + metaData.getTables("druid", "dr_id", CalciteTests.SOME_DATSOURCE_ESCAPED, null), + ImmutableSet.of("TABLE_SCHEM", "TABLE_NAME") + ) + ); + + Assert.assertEquals( + ImmutableList.of( + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE) + ) + ), + getRows( + metaData.getTables("druid", "dr_id", CalciteTests.SOMEXDATASOURCE, null), + ImmutableSet.of("TABLE_SCHEM", "TABLE_NAME") + ) + ); + + // Assert that some_datasource is treated as a pattern that matches some_datasource and somexdatasource + Assert.assertEquals( + ImmutableList.of( + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE) + ), + row( + Pair.of("TABLE_SCHEM", "druid"), + Pair.of("TABLE_NAME", CalciteTests.SOMEXDATASOURCE) + ) + ), + getRows( + metaData.getTables("druid", "dr_id", CalciteTests.SOME_DATASOURCE, null), + ImmutableSet.of("TABLE_SCHEM", "TABLE_NAME") + ) + ); + } + private static List> getRows(final ResultSet resultSet) throws SQLException { return getRows(resultSet, null); 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 4fc1e8e92c87..7c475e5fe037 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 @@ -58,6 +58,7 @@ import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; +import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.join.JoinType; @@ -102,11 +103,19 @@ public class BaseCalciteQueryTest extends CalciteTestBase { - public static final String NULL_STRING = NullHandling.defaultStringValue(); - public static final Float NULL_FLOAT = NullHandling.defaultFloatValue(); - public static final Long NULL_LONG = NullHandling.defaultLongValue(); + public static String NULL_STRING; + public static Float NULL_FLOAT; + public static Long NULL_LONG; public static final String HLLC_STRING = VersionOneHyperLogLogCollector.class.getName(); + @BeforeClass + public static void setupNullValues() + { + NULL_STRING = NullHandling.defaultStringValue(); + NULL_FLOAT = NullHandling.defaultFloatValue(); + NULL_LONG = NullHandling.defaultLongValue(); + } + public static final Logger log = new Logger(BaseCalciteQueryTest.class); public static final PlannerConfig PLANNER_CONFIG_DEFAULT = new PlannerConfig(); @@ -154,7 +163,7 @@ public DateTimeZone getSqlTimeZone() public static final String DUMMY_SQL_ID = "dummy"; public static final String LOS_ANGELES = "America/Los_Angeles"; - static final ImmutableMap.Builder DEFAULT_QUERY_CONTEXT_BUILDER = + private static final ImmutableMap.Builder DEFAULT_QUERY_CONTEXT_BUILDER = ImmutableMap.builder() .put(PlannerContext.CTX_SQL_QUERY_ID, DUMMY_SQL_ID) .put(PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z") @@ -195,6 +204,16 @@ public DateTimeZone getSqlTimeZone() QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE ); + // Add additional context to the given context map for when the + // timeseries query has timestamp_floor expression on the timestamp dimension + public static Map getTimeseriesContextWithFloorTime(Map context, + String timestampResultField) + { + return ImmutableMap.builder().putAll(context) + .put(TimeseriesQuery.CTX_TIMESTAMP_RESULT_FIELD, timestampResultField) + .build(); + } + // Matches QUERY_CONTEXT_LOS_ANGELES public static final Map TIMESERIES_CONTEXT_LOS_ANGELES = new HashMap<>(); @@ -403,7 +422,10 @@ public QueryLogHook getQueryLogHook() @Before public void setUp() throws Exception { - walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder()); + walker = CalciteTests.createMockWalker( + conglomerate, + temporaryFolder.newFolder() + ); } @After 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 81cf68d68927..622842fd32eb 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 @@ -22,12 +22,15 @@ import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import junitparams.JUnitParamsRunner; +import junitparams.Parameters; +import org.apache.calcite.plan.RelOptPlanner; import org.apache.calcite.runtime.CalciteContextException; import org.apache.calcite.tools.ValidationException; +import org.apache.druid.annotations.UsedByJUnitParamsRunner; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; -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.java.util.common.StringUtils; @@ -35,9 +38,11 @@ import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.Druids; +import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.LookupDataSource; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryDataSource; +import org.apache.druid.query.QueryException; import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; @@ -76,6 +81,8 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.LikeDimFilter; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.RegexDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -105,6 +112,7 @@ import org.junit.Ignore; import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; +import org.junit.runner.RunWith; import java.util.ArrayList; import java.util.Collections; @@ -112,6 +120,7 @@ import java.util.List; import java.util.Map; +@RunWith(JUnitParamsRunner.class) public class CalciteQueryTest extends BaseCalciteQueryTest { private final boolean useDefault = NullHandling.replaceWithDefault(); @@ -130,6 +139,30 @@ public void testSelectConstantExpression() throws Exception ); } + @Test + public void testSelectNonNumericNumberLiterals() throws Exception + { + // Tests to convert NaN, positive infinity and negative infinity as literals. + testQuery( + "SELECT" + + " CAST(1 / 0.0 AS BIGINT)," + + " CAST(1 / -0.0 AS BIGINT)," + + " CAST(-1 / 0.0 AS BIGINT)," + + " CAST(-1 / -0.0 AS BIGINT)," + + " CAST(0/ 0.0 AS BIGINT)", + ImmutableList.of(), + ImmutableList.of( + new Object[] { + Long.MAX_VALUE, + Long.MAX_VALUE, + Long.MIN_VALUE, + Long.MIN_VALUE, + 0L + } + ) + ); + } + @Test public void testSelectConstantExpressionFromTable() throws Exception { @@ -152,6 +185,344 @@ public void testSelectConstantExpressionFromTable() throws Exception ); } + @Test + public void testJoinOuterGroupByAndSubqueryHasLimit() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "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", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns(ImmutableList.of("m1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("m1"), + DruidExpression.fromColumn("j0.m1") + ), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .limit(10) + .columns("__time", "cnt", "dim1", "dim2", "dim3", "j0.m1", "m1", "m2", "unique_dim1") + .context(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators( + new DoubleSumAggregatorFactory("a0:sum", "m2"), + new CountAggregatorFactory("a0:count") + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "a0:sum"), + new FieldAccessPostAggregator(null, "a0:count") + ) + ) + + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + NullHandling.sqlCompatible() + ? ImmutableList.of( + new Object[]{null, 4.0}, + new Object[]{"", 3.0}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ) + : ImmutableList.of( + new Object[]{"", 3.6666666666666665}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ) + ); + } + + @Test + public void testJoinOuterGroupByAndSubqueryNoLimit() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT dim2, AVG(m2) FROM (SELECT * FROM foo AS t1 INNER JOIN foo AS t2 ON t1.m1 = t2.m1) AS t3 GROUP BY dim2", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns(ImmutableList.of("m1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("m1"), + DruidExpression.fromColumn("j0.m1") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators( + new DoubleSumAggregatorFactory("a0:sum", "m2"), + new CountAggregatorFactory("a0:count") + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "a0:sum"), + new FieldAccessPostAggregator(null, "a0:count") + ) + ) + + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + NullHandling.sqlCompatible() + ? ImmutableList.of( + new Object[]{null, 4.0}, + new Object[]{"", 3.0}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ) + : ImmutableList.of( + new Object[]{"", 3.6666666666666665}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ) + ); + } + + @Test + public void testJoinWithLimitBeforeJoining() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT t1.dim2, AVG(t1.m2) FROM (SELECT * FROM foo LIMIT 10) AS t1 INNER JOIN foo AS t2 ON t1.m1 = t2.m1 GROUP BY t1.dim2", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("dim2", "m1", "m2") + .context(QUERY_CONTEXT_DEFAULT) + .limit(10) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns(ImmutableList.of("m1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("m1"), + DruidExpression.fromColumn("j0.m1") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions(new DefaultDimensionSpec("dim2", "d0", ValueType.STRING)) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators( + new DoubleSumAggregatorFactory("a0:sum", "m2"), + new CountAggregatorFactory("a0:count") + ) + ) + .setPostAggregatorSpecs( + ImmutableList.of( + new ArithmeticPostAggregator( + "a0", + "quotient", + ImmutableList.of( + new FieldAccessPostAggregator(null, "a0:sum"), + new FieldAccessPostAggregator(null, "a0:count") + ) + ) + + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + NullHandling.sqlCompatible() + ? ImmutableList.of( + new Object[]{null, 4.0}, + new Object[]{"", 3.0}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ) + : ImmutableList.of( + new Object[]{"", 3.6666666666666665}, + new Object[]{"a", 2.5}, + new Object[]{"abc", 5.0} + ) + ); + } + + @Test + public void testJoinOnTimeseriesWithFloorOnTime() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT CAST(__time AS BIGINT), m1, ANY_VALUE(dim3, 100) FROM foo WHERE (TIME_FLOOR(__time, 'PT1H'), m1) IN\n" + + " (\n" + + " SELECT TIME_FLOOR(__time, 'PT1H') AS t1, MIN(m1) AS t2 FROM foo WHERE dim3 = 'b'\n" + + " AND __time BETWEEN '1994-04-29 00:00:00' AND '2020-01-11 00:00:00' GROUP BY 1\n" + + " )\n" + + "GROUP BY 1, 2\n", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Intervals.of("1994-04-29/2020-01-11T00:00:00.001Z"))) + .filters(selector("dim3", "b", null)) + .granularity(new PeriodGranularity(Period.hours(1), null, DateTimeZone.UTC)) + .aggregators(aggregators( + new FloatMinAggregatorFactory("a0", "m1") + )) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) + .build()), + "j0.", + "((timestamp_floor(\"__time\",'PT1H',null,'UTC') == \"j0.d0\") && (\"m1\" == \"j0.a0\"))", + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions( + new DefaultDimensionSpec("__time", "d0", ValueType.LONG), + new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT) + + ) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators( + new StringAnyAggregatorFactory("a0", "dim3", 100) + )) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{946684800000L, 1.0f, "[a, b]"}, + new Object[]{946771200000L, 2.0f, "[b, c]"} + ) + ); + } + + @Test + public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime() throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT CAST(__time AS BIGINT), m1, ANY_VALUE(dim3, 100) FROM foo WHERE (CAST(TIME_FLOOR(__time, 'PT1H') AS BIGINT), m1) IN\n" + + " (\n" + + " SELECT CAST(TIME_FLOOR(__time, 'PT1H') AS BIGINT) + 0 AS t1, MIN(m1) AS t2 FROM foo WHERE dim3 = 'b'\n" + + " AND __time BETWEEN '1994-04-29 00:00:00' AND '2020-01-11 00:00:00' GROUP BY 1\n" + + " )\n" + + "GROUP BY 1, 2\n", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Intervals.of("1994-04-29/2020-01-11T00:00:00.001Z"))) + .setVirtualColumns( + expressionVirtualColumn( + "v0", + "(timestamp_floor(\"__time\",'PT1H',null,'UTC') + 0)", + ValueType.LONG + ) + ) + .setDimFilter(selector("dim3", "b", null)) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ValueType.LONG))) + .setAggregatorSpecs(aggregators( + new FloatMinAggregatorFactory("a0", "m1") + )) + .setContext(QUERY_CONTEXT_DEFAULT) + .build()), + "j0.", + "((timestamp_floor(\"__time\",'PT1H',null,'UTC') == \"j0.d0\") && (\"m1\" == \"j0.a0\"))", + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions( + new DefaultDimensionSpec("__time", "d0", ValueType.LONG), + new DefaultDimensionSpec("m1", "d1", ValueType.FLOAT) + + ) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators( + new StringAnyAggregatorFactory("a0", "dim3", 100) + )) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{946684800000L, 1.0f, "[a, b]"}, + new Object[]{946771200000L, 2.0f, "[b, c]"} + ) + ); + } @Test public void testSelectCountStart() throws Exception @@ -215,7 +586,7 @@ public void testSelectCountStart() throws Exception .aggregators(aggregators( new CountAggregatorFactory("a0") )) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build()), ImmutableList.of() ); @@ -337,55 +708,61 @@ public void testInformationSchemaSchemata() throws Exception public void testInformationSchemaTables() throws Exception { testQuery( - "SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE\n" + "SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, IS_JOINABLE, IS_BROADCAST\n" + "FROM INFORMATION_SCHEMA.TABLES\n" + "WHERE TABLE_TYPE IN ('SYSTEM_TABLE', 'TABLE', 'VIEW')", ImmutableList.of(), ImmutableList.builder() - .add(new Object[]{"druid", CalciteTests.DATASOURCE1, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE2, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE4, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE5, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE"}) - .add(new Object[]{"druid", "aview", "VIEW"}) - .add(new Object[]{"druid", "bview", "VIEW"}) - .add(new Object[]{"INFORMATION_SCHEMA", "COLUMNS", "SYSTEM_TABLE"}) - .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE"}) - .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE"}) - .add(new Object[]{"lookup", "lookyloo", "TABLE"}) - .add(new Object[]{"sys", "segments", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "server_segments", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "servers", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "supervisors", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "tasks", "SYSTEM_TABLE"}) + .add(new Object[]{"druid", CalciteTests.BROADCAST_DATASOURCE, "TABLE", "YES", "YES"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE1, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE2, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE4, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE5, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.SOME_DATASOURCE, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.SOMEXDATASOURCE, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", "aview", "VIEW", "NO", "NO"}) + .add(new Object[]{"druid", "bview", "VIEW", "NO", "NO"}) + .add(new Object[]{"INFORMATION_SCHEMA", "COLUMNS", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"lookup", "lookyloo", "TABLE", "YES", "YES"}) + .add(new Object[]{"sys", "segments", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "server_segments", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "servers", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "supervisors", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "tasks", "SYSTEM_TABLE", "NO", "NO"}) .build() ); testQuery( PLANNER_CONFIG_DEFAULT, - "SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE\n" + "SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, IS_JOINABLE, IS_BROADCAST\n" + "FROM INFORMATION_SCHEMA.TABLES\n" + "WHERE TABLE_TYPE IN ('SYSTEM_TABLE', 'TABLE', 'VIEW')", CalciteTests.SUPER_USER_AUTH_RESULT, ImmutableList.of(), ImmutableList.builder() - .add(new Object[]{"druid", CalciteTests.DATASOURCE1, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE2, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE4, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.FORBIDDEN_DATASOURCE, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE5, "TABLE"}) - .add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE"}) - .add(new Object[]{"druid", "aview", "VIEW"}) - .add(new Object[]{"druid", "bview", "VIEW"}) - .add(new Object[]{"INFORMATION_SCHEMA", "COLUMNS", "SYSTEM_TABLE"}) - .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE"}) - .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE"}) - .add(new Object[]{"lookup", "lookyloo", "TABLE"}) - .add(new Object[]{"sys", "segments", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "server_segments", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "servers", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "supervisors", "SYSTEM_TABLE"}) - .add(new Object[]{"sys", "tasks", "SYSTEM_TABLE"}) + .add(new Object[]{"druid", CalciteTests.BROADCAST_DATASOURCE, "TABLE", "YES", "YES"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE1, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE2, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE4, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.FORBIDDEN_DATASOURCE, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE5, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.DATASOURCE3, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.SOME_DATASOURCE, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", CalciteTests.SOMEXDATASOURCE, "TABLE", "NO", "NO"}) + .add(new Object[]{"druid", "aview", "VIEW", "NO", "NO"}) + .add(new Object[]{"druid", "bview", "VIEW", "NO", "NO"}) + .add(new Object[]{"INFORMATION_SCHEMA", "COLUMNS", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"INFORMATION_SCHEMA", "SCHEMATA", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"INFORMATION_SCHEMA", "TABLES", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"lookup", "lookyloo", "TABLE", "YES", "YES"}) + .add(new Object[]{"sys", "segments", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "server_segments", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "servers", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "supervisors", "SYSTEM_TABLE", "NO", "NO"}) + .add(new Object[]{"sys", "tasks", "SYSTEM_TABLE", "NO", "NO"}) .build() ); } @@ -1783,6 +2160,23 @@ public void testFirstLatestAggregatorsSkipNulls() throws Exception ); } + // This query is expected to fail as we do not support join on multi valued column + // (see issue https://github.com/apache/druid/issues/9924 for more information) + // TODO: Remove expected Exception when https://github.com/apache/druid/issues/9924 is fixed + @Test(expected = QueryException.class) + @Parameters(source = QueryContextForJoinProvider.class) + public void testJoinOnMultiValuedColumnShouldThrowException(Map queryContext) throws Exception + { + final String query = "SELECT dim3, l.v from druid.foo f inner join lookup.lookyloo l on f.dim3 = l.k\n"; + + testQuery( + query, + queryContext, + ImmutableList.of(), + ImmutableList.of() + ); + } + @Test public void testAnyAggregatorsDoesNotSkipNulls() throws Exception { @@ -2381,7 +2775,7 @@ public void testGroupByExpressionAliasedAsOriginalColumnName() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.MONTH) .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -6378,26 +6772,14 @@ public void testMinMaxAvgDailyCountWithLimit() throws Exception GroupByQuery.builder() .setDataSource( new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - expressionVirtualColumn( - "v0", - "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG - ) - ) - .setDimensions(dimensions(new DefaultDimensionSpec( - "v0", - "d0", - ValueType.LONG - ))) - .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() - ) + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) + .intervals(querySegmentSpec(Filtration.eternity())) + .aggregators(new CountAggregatorFactory("a0")) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) + .build() + ) ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) @@ -6444,44 +6826,30 @@ public void testAvgDailyCountDistinct() throws Exception GroupByQuery.builder() .setDataSource( new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setVirtualColumns( - expressionVirtualColumn( - "v0", - "timestamp_floor(\"__time\",'P1D',null,'UTC')", - ValueType.LONG - ) - ) - .setDimensions(dimensions(new DefaultDimensionSpec( - "v0", - "d0", - ValueType.LONG - ))) - .setAggregatorSpecs( - aggregators( - new CardinalityAggregatorFactory( - "a0:a", - null, - dimensions(new DefaultDimensionSpec( - "cnt", - "cnt", - ValueType.LONG - )), - false, - true - ) - ) - ) - .setPostAggregatorSpecs( - ImmutableList.of( - new HyperUniqueFinalizingPostAggregator("a0", "a0:a") - ) - ) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) + .aggregators( + new CardinalityAggregatorFactory( + "a0:a", + null, + dimensions(new DefaultDimensionSpec( + "cnt", + "cnt", + ValueType.LONG + )), + false, + true + ) + ) + .postAggregators( + ImmutableList.of( + new HyperUniqueFinalizingPostAggregator("a0", "a0:a") + ) + ) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) + .build() ) ) .setInterval(querySegmentSpec(Filtration.eternity())) @@ -6510,7 +6878,8 @@ public void testAvgDailyCountDistinct() throws Exception } @Test - public void testTopNFilterJoin() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testTopNFilterJoin(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -6530,6 +6899,7 @@ public void testTopNFilterJoin() throws Exception + ") t2 ON (t1.dim2 = t2.dim2)\n" + "GROUP BY t1.dim1\n" + "ORDER BY 1\n", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -6571,7 +6941,7 @@ public void testTopNFilterJoin() throws Exception Integer.MAX_VALUE ) ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -6582,7 +6952,8 @@ public void testTopNFilterJoin() throws Exception } @Test - public void testTopNFilterJoinWithProjection() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testTopNFilterJoinWithProjection(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -6602,6 +6973,7 @@ public void testTopNFilterJoinWithProjection() throws Exception + " LIMIT 2\n" + ") t2 ON (t1.dim2 = t2.dim2)\n" + "GROUP BY SUBSTRING(t1.dim1, 1, 10)", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -6640,7 +7012,7 @@ public void testTopNFilterJoinWithProjection() throws Exception ) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -6651,8 +7023,9 @@ public void testTopNFilterJoinWithProjection() throws Exception } @Test + @Parameters(source = QueryContextForJoinProvider.class) @Ignore("Stopped working after the ability to join on subqueries was added to DruidJoinRule") - public void testRemovableLeftJoin() throws Exception + public void testRemovableLeftJoin(Map queryContext) throws Exception { // LEFT JOIN where the right-hand side can be ignored. @@ -6670,6 +7043,7 @@ public void testRemovableLeftJoin() throws Exception + ") t2 ON (t1.dim2 = t2.dim2)\n" + "GROUP BY t1.dim1\n" + "ORDER BY 1\n", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) @@ -6689,7 +7063,7 @@ public void testRemovableLeftJoin() throws Exception Integer.MAX_VALUE ) ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -6903,6 +7277,9 @@ public void testExactCountDistinctUsingSubqueryWithWherePushDown() throws Except @Test public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() throws Exception { + // Cannot vectorize topN operator. + cannotVectorize(); + testQuery( "SELECT\n" + " SUM(cnt),\n" @@ -6913,15 +7290,16 @@ public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() throws E GroupByQuery.builder() .setDataSource( new QueryDataSource( - GroupByQuery.builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) - .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setLimit(1) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + new TopNQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .dimension(new DefaultDimensionSpec("dim2", "d0")) + .aggregators(new LongSumAggregatorFactory("a0", "cnt")) + .metric(new DimensionTopNMetricSpec(null, StringComparators.LEXICOGRAPHIC)) + .threshold(1) + .context(QUERY_CONTEXT_DEFAULT) + .build() ) ) .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC)) @@ -7300,6 +7678,74 @@ public void testRegexpExtract() throws Exception ); } + @Test + public void testRegexpExtractFilterViaNotNullCheck() throws Exception + { + // Cannot vectorize due to extractionFn in dimension spec. + cannotVectorize(); + + testQuery( + "SELECT COUNT(*)\n" + + "FROM foo\n" + + "WHERE REGEXP_EXTRACT(dim1, '^1') IS NOT NULL OR REGEXP_EXTRACT('Z' || dim1, '^Z2') IS NOT NULL", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .virtualColumns( + expressionVirtualColumn("v0", "regexp_extract(concat('Z',\"dim1\"),'^Z2')", ValueType.STRING) + ) + .filters( + or( + not(selector("dim1", null, new RegexDimExtractionFn("^1", 0, true, null))), + not(selector("v0", null, null)) + ) + ) + .aggregators(new CountAggregatorFactory("a0")) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{3L} + ) + ); + } + + @Test + public void testRegexpLikeFilter() throws Exception + { + // Cannot vectorize due to usage of regex filter. + cannotVectorize(); + + testQuery( + "SELECT COUNT(*)\n" + + "FROM foo\n" + + "WHERE REGEXP_LIKE(dim1, '^1') OR REGEXP_LIKE('Z' || dim1, '^Z2')", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .virtualColumns( + expressionVirtualColumn("v0", "concat('Z',\"dim1\")", ValueType.STRING) + ) + .filters( + or( + new RegexDimFilter("dim1", "^1", null), + new RegexDimFilter("v0", "^Z2", null) + ) + ) + .aggregators(new CountAggregatorFactory("a0")) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{3L} + ) + ); + } + @Test public void testGroupBySortPushDown() throws Exception { @@ -7477,7 +7923,7 @@ public void testGroupAndFilterOnTimeFloorWithTimeZone() throws Exception .intervals(querySegmentSpec(Intervals.of("2000-01-01T00-08:00/2000-03-01T00-08:00"))) .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES))) .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -8075,7 +8521,44 @@ public void testFilterAndGroupByLookup() throws Exception } @Test - public void testFilterAndGroupByLookupUsingJoinOperatorAllowNulls() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMatchesNothig(Map queryContext) throws Exception + { + // Cannot vectorize JOIN operator. + cannotVectorize(); + + testQuery( + "SELECT lookyloo.k, COUNT(*)\n" + + "FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k\n" + + "WHERE lookyloo.v = '123'\n" + + "GROUP BY lookyloo.k", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim2"), DruidExpression.fromColumn("j0.k")), + JoinType.LEFT + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimFilter(selector("j0.v", "123", null)) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("j0.k", "d0"))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(queryContext) + .build() + ), + ImmutableList.of() + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testFilterAndGroupByLookupUsingJoinOperatorAllowNulls(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -8085,6 +8568,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorAllowNulls() throws Excep + "FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k\n" + "WHERE lookyloo.v <> 'xa' OR lookyloo.v IS NULL\n" + "GROUP BY lookyloo.v", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -8101,7 +8585,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorAllowNulls() throws Excep .setDimFilter(or(not(selector("j0.v", "xa", null)), selector("j0.v", null, null))) .setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -8112,7 +8596,8 @@ public void testFilterAndGroupByLookupUsingJoinOperatorAllowNulls() throws Excep } @Test - public void testFilterAndGroupByLookupUsingJoinOperatorBackwards() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testFilterAndGroupByLookupUsingJoinOperatorBackwards(Map queryContext) throws Exception { // Like "testFilterAndGroupByLookupUsingJoinOperator", but with the table and lookup reversed. @@ -8124,6 +8609,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorBackwards() throws Except + "FROM lookup.lookyloo RIGHT JOIN foo ON foo.dim2 = lookyloo.k\n" + "WHERE lookyloo.v <> 'xa'\n" + "GROUP BY lookyloo.v", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -8147,7 +8633,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorBackwards() throws Except .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("v", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -8158,7 +8644,8 @@ public void testFilterAndGroupByLookupUsingJoinOperatorBackwards() throws Except } @Test - public void testFilterAndGroupByLookupUsingJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testFilterAndGroupByLookupUsingJoinOperatorWithNotFilter(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -8168,6 +8655,7 @@ public void testFilterAndGroupByLookupUsingJoinOperator() throws Exception + "FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k\n" + "WHERE lookyloo.v <> 'xa'\n" + "GROUP BY lookyloo.v", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -8184,7 +8672,7 @@ public void testFilterAndGroupByLookupUsingJoinOperator() throws Exception .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -8195,19 +8683,18 @@ public void testFilterAndGroupByLookupUsingJoinOperator() throws Exception } @Test - public void testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdown() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testFilterAndGroupByLookupUsingJoinOperator(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); - Map queryRewriteValueColumnFiltersContext = DEFAULT_QUERY_CONTEXT_BUILDER - .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true) - .build(); + testQuery( "SELECT lookyloo.k, COUNT(*)\n" + "FROM foo LEFT JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k\n" + "WHERE lookyloo.v = 'xa'\n" + "GROUP BY lookyloo.k", - queryRewriteValueColumnFiltersContext, + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -8224,7 +8711,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdown() .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("j0.k", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(queryRewriteValueColumnFiltersContext) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -8234,7 +8721,8 @@ public void testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdown() } @Test - public void testFilterAndGroupByLookupUsingPostAggregationJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testFilterAndGroupByLookupUsingPostAggregationJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT base.dim2, lookyloo.v, base.cnt FROM (\n" @@ -8242,6 +8730,7 @@ public void testFilterAndGroupByLookupUsingPostAggregationJoinOperator() throws + ") base\n" + "LEFT JOIN lookup.lookyloo ON base.dim2 = lookyloo.k\n" + "WHERE lookyloo.v <> 'xa' OR lookyloo.v IS NULL", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8254,7 +8743,7 @@ public void testFilterAndGroupByLookupUsingPostAggregationJoinOperator() throws .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), new LookupDataSource("lookyloo"), @@ -8282,7 +8771,8 @@ public void testFilterAndGroupByLookupUsingPostAggregationJoinOperator() throws } @Test - public void testGroupByInnerJoinOnLookupUsingJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testGroupByInnerJoinOnLookupUsingJoinOperator(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -8291,6 +8781,7 @@ public void testGroupByInnerJoinOnLookupUsingJoinOperator() throws Exception "SELECT lookyloo.v, COUNT(*)\n" + "FROM foo INNER JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k\n" + "GROUP BY lookyloo.v", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -8306,7 +8797,7 @@ public void testGroupByInnerJoinOnLookupUsingJoinOperator() throws Exception .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -8316,11 +8807,13 @@ public void testGroupByInnerJoinOnLookupUsingJoinOperator() throws Exception } @Test - public void testSelectOnLookupUsingInnerJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testSelectOnLookupUsingInnerJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT dim2, lookyloo.*\n" + "FROM foo INNER JOIN lookup.lookyloo ON foo.dim2 = lookyloo.k\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8334,7 +8827,7 @@ public void testSelectOnLookupUsingInnerJoinOperator() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim2", "j0.k", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8346,13 +8839,15 @@ public void testSelectOnLookupUsingInnerJoinOperator() throws Exception } @Test - public void testLeftJoinTwoLookupsUsingJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testLeftJoinTwoLookupsUsingJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT dim1, dim2, l1.v, l2.v\n" + "FROM foo\n" + "LEFT JOIN lookup.lookyloo l1 ON foo.dim1 = l1.k\n" + "LEFT JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8372,7 +8867,7 @@ public void testLeftJoinTwoLookupsUsingJoinOperator() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("_j0.v", "dim1", "dim2", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8387,7 +8882,8 @@ public void testLeftJoinTwoLookupsUsingJoinOperator() throws Exception } @Test - public void testInnerJoinTableLookupLookupWithFilterWithOuterLimit() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinTableLookupLookupWithFilterWithOuterLimit(Map queryContext) throws Exception { testQuery( "SELECT dim1\n" @@ -8396,6 +8892,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithOuterLimit() throws Exce + "INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k\n" + "WHERE l.v = 'xa'\n" + "LIMIT 100\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8417,7 +8914,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithOuterLimit() throws Exce .limit(100) .filters(selector("j0.v", "xa", null)) .columns("dim1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8428,7 +8925,8 @@ public void testInnerJoinTableLookupLookupWithFilterWithOuterLimit() throws Exce } @Test - public void testInnerJoinTableLookupLookupWithFilterWithoutLimit() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinTableLookupLookupWithFilterWithoutLimit(Map queryContext) throws Exception { testQuery( "SELECT dim1\n" @@ -8436,6 +8934,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithoutLimit() throws Except + "INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k\n" + "INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k\n" + "WHERE l.v = 'xa'\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8456,7 +8955,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithoutLimit() throws Except .intervals(querySegmentSpec(Filtration.eternity())) .filters(selector("j0.v", "xa", null)) .columns("dim1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8467,7 +8966,8 @@ public void testInnerJoinTableLookupLookupWithFilterWithoutLimit() throws Except } @Test - public void testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns(Map queryContext) throws Exception { testQuery( "SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1\n" @@ -8476,6 +8976,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns + "INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k\n" + "WHERE l.v = 'xa'\n" + "LIMIT 100\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8497,7 +8998,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns .limit(100) .filters(selector("j0.v", "xa", null)) .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8508,7 +9009,8 @@ public void testInnerJoinTableLookupLookupWithFilterWithOuterLimitWithAllColumns } @Test - public void testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns(Map queryContext) throws Exception { testQuery( "SELECT __time, cnt, dim1, dim2, dim3, m1, m2, unique_dim1\n" @@ -8516,6 +9018,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns() + "INNER JOIN lookup.lookyloo l ON foo.dim2 = l.k\n" + "INNER JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k\n" + "WHERE l.v = 'xa'\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8536,7 +9039,7 @@ public void testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns() .intervals(querySegmentSpec(Filtration.eternity())) .filters(selector("j0.v", "xa", null)) .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8547,7 +9050,8 @@ public void testInnerJoinTableLookupLookupWithFilterWithoutLimitWithAllColumns() } @Test - public void testManyManyInnerJoinOnManyManyLookup() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testManyManyInnerJoinOnManyManyLookup(Map queryContext) throws Exception { testQuery( "SELECT dim1\n" @@ -8572,6 +9076,7 @@ public void testManyManyInnerJoinOnManyManyLookup() throws Exception + "INNER JOIN lookup.lookyloo l18 ON foo.dim2 = l18.k\n" + "INNER JOIN lookup.lookyloo l19 ON foo.dim2 = l19.k\n" + "WHERE l.v = 'xa'\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8694,7 +9199,7 @@ public void testManyManyInnerJoinOnManyManyLookup() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .filters(selector("j0.v", "xa", null)) .columns("dim1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8705,7 +9210,8 @@ public void testManyManyInnerJoinOnManyManyLookup() throws Exception } @Test - public void testInnerJoinQueryOfLookup() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinQueryOfLookup(Map queryContext) throws Exception { // Cannot vectorize the subquery. cannotVectorize(); @@ -8716,6 +9222,7 @@ public void testInnerJoinQueryOfLookup() throws Exception + "INNER JOIN \n" + " (SELECT SUBSTRING(k, 1, 1) k, LATEST(v, 10) v FROM lookup.lookyloo GROUP BY 1) t1\n" + " ON foo.dim2 = t1.k", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8744,7 +9251,7 @@ public void testInnerJoinQueryOfLookup() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1", "dim2", "j0.a0") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8755,7 +9262,8 @@ public void testInnerJoinQueryOfLookup() throws Exception } @Test - public void testInnerJoinQueryOfLookupRemovable() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinQueryOfLookupRemovable(Map queryContext) throws Exception { // Like "testInnerJoinQueryOfLookup", but the subquery is removable. @@ -8765,6 +9273,7 @@ public void testInnerJoinQueryOfLookupRemovable() throws Exception + "INNER JOIN \n" + " (SELECT k, SUBSTRING(v, 1, 3) sk FROM lookup.lookyloo) t1\n" + " ON foo.dim2 = t1.k", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -8779,7 +9288,7 @@ public void testInnerJoinQueryOfLookupRemovable() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns(expressionVirtualColumn("v0", "substring(\"j0.v\", 0, 3)", ValueType.STRING)) .columns("dim1", "dim2", "v0") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -8791,7 +9300,8 @@ public void testInnerJoinQueryOfLookupRemovable() throws Exception } @Test - public void testInnerJoinTwoLookupsToTableUsingNumericColumn() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinTwoLookupsToTableUsingNumericColumn(Map queryContext) throws Exception { // Regression test for https://github.com/apache/druid/issues/9646. @@ -8803,6 +9313,7 @@ public void testInnerJoinTwoLookupsToTableUsingNumericColumn() throws Exception + "FROM foo\n" + "INNER JOIN lookup.lookyloo l1 ON l1.k = foo.m1\n" + "INNER JOIN lookup.lookyloo l2 ON l2.k = l1.k", + queryContext, ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource( @@ -8850,7 +9361,8 @@ public void testInnerJoinTwoLookupsToTableUsingNumericColumn() throws Exception } @Test - public void testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse(Map queryContext) throws Exception { // Like "testInnerJoinTwoLookupsToTableUsingNumericColumn", but the tables are specified backwards. @@ -8861,6 +9373,7 @@ public void testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse() throws E + "FROM lookup.lookyloo l1\n" + "INNER JOIN lookup.lookyloo l2 ON l1.k = l2.k\n" + "INNER JOIN foo on l2.k = foo.m1", + queryContext, ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource( @@ -8904,7 +9417,8 @@ public void testInnerJoinTwoLookupsToTableUsingNumericColumnInReverse() throws E } @Test - public void testInnerJoinLookupTableTable() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinLookupTableTable(Map queryContext) throws Exception { // Regression test for https://github.com/apache/druid/issues/9646. @@ -8917,6 +9431,7 @@ public void testInnerJoinLookupTableTable() throws Exception + "INNER JOIN druid.foo f on f.dim1 = l.k\n" + "INNER JOIN druid.numfoo nf on nf.dim1 = l.k\n" + "GROUP BY 1, 2 ORDER BY 2", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -8974,7 +9489,7 @@ public void testInnerJoinLookupTableTable() throws Exception null ) ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -8984,7 +9499,8 @@ public void testInnerJoinLookupTableTable() throws Exception } @Test - public void testInnerJoinLookupTableTableChained() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinLookupTableTableChained(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -8995,6 +9511,7 @@ public void testInnerJoinLookupTableTableChained() throws Exception + "INNER JOIN druid.foo f on f.dim1 = l.k\n" + "INNER JOIN druid.numfoo nf on nf.dim1 = f.dim1\n" + "GROUP BY 1, 2 ORDER BY 2", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -9052,7 +9569,7 @@ public void testInnerJoinLookupTableTableChained() throws Exception null ) ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -9087,7 +9604,7 @@ public void testWhereInSelectNullFromLookup() throws Exception .build() ), "j0.", - "0", + equalsCondition(DruidExpression.fromColumn("dim1"), DruidExpression.fromColumn("j0.d0")), JoinType.INNER ) ) @@ -9141,7 +9658,8 @@ public void testCommaJoinLeftFunction() throws Exception // Unfortunately, we have disabled pushing down predicates (conditions and filters) due to https://github.com/apache/druid/pull/9773 // Hence, comma join will result in a cross join with filter on outermost @Test - public void testCommaJoinTableLookupTableMismatchedTypes() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testCommaJoinTableLookupTableMismatchedTypes(Map queryContext) throws Exception { // Regression test for https://github.com/apache/druid/issues/9646. @@ -9152,6 +9670,7 @@ public void testCommaJoinTableLookupTableMismatchedTypes() throws Exception "SELECT COUNT(*)\n" + "FROM foo, lookup.lookyloo l, numfoo\n" + "WHERE foo.cnt = l.k AND l.k = numfoo.cnt\n", + queryContext, ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource( @@ -9192,7 +9711,8 @@ public void testCommaJoinTableLookupTableMismatchedTypes() throws Exception } @Test - public void testJoinTableLookupTableMismatchedTypesWithoutComma() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -9202,6 +9722,7 @@ public void testJoinTableLookupTableMismatchedTypesWithoutComma() throws Excepti + "FROM foo\n" + "INNER JOIN lookup.lookyloo l ON foo.cnt = l.k\n" + "INNER JOIN numfoo ON l.k = numfoo.cnt\n", + queryContext, ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource( @@ -9217,7 +9738,7 @@ public void testJoinTableLookupTableMismatchedTypesWithoutComma() throws Excepti ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("k", "v0") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), "j0.", @@ -9233,7 +9754,7 @@ public void testJoinTableLookupTableMismatchedTypesWithoutComma() throws Excepti .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("cnt") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), "_j0.", @@ -9255,7 +9776,8 @@ public void testJoinTableLookupTableMismatchedTypesWithoutComma() throws Excepti } @Test - public void testInnerJoinCastLeft() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinCastLeft(Map queryContext) throws Exception { // foo.m1 is FLOAT, l.k is STRING. @@ -9263,6 +9785,7 @@ public void testInnerJoinCastLeft() throws Exception "SELECT foo.m1, l.k, l.v\n" + "FROM foo\n" + "INNER JOIN lookup.lookyloo l ON CAST(foo.m1 AS VARCHAR) = l.k\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9279,7 +9802,7 @@ public void testInnerJoinCastLeft() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.k", "j0.v", "m1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of() @@ -9287,7 +9810,8 @@ public void testInnerJoinCastLeft() throws Exception } @Test - public void testInnerJoinCastRight() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinCastRight(Map queryContext) throws Exception { // foo.m1 is FLOAT, l.k is STRING. @@ -9295,6 +9819,7 @@ public void testInnerJoinCastRight() throws Exception "SELECT foo.m1, l.k, l.v\n" + "FROM foo\n" + "INNER JOIN lookup.lookyloo l ON foo.m1 = CAST(l.k AS FLOAT)\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9309,7 +9834,7 @@ public void testInnerJoinCastRight() throws Exception ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("k", "v", "v0") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), "j0.", @@ -9319,7 +9844,7 @@ public void testInnerJoinCastRight() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.k", "j0.v", "m1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9329,7 +9854,8 @@ public void testInnerJoinCastRight() throws Exception } @Test - public void testInnerJoinMismatchedTypes() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinMismatchedTypes(Map queryContext) throws Exception { // foo.m1 is FLOAT, l.k is STRING. Comparing them generates a CAST. @@ -9337,6 +9863,7 @@ public void testInnerJoinMismatchedTypes() throws Exception "SELECT foo.m1, l.k, l.v\n" + "FROM foo\n" + "INNER JOIN lookup.lookyloo l ON foo.m1 = l.k\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9351,7 +9878,7 @@ public void testInnerJoinMismatchedTypes() throws Exception ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("k", "v", "v0") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), "j0.", @@ -9361,7 +9888,7 @@ public void testInnerJoinMismatchedTypes() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("j0.k", "j0.v", "m1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9371,12 +9898,14 @@ public void testInnerJoinMismatchedTypes() throws Exception } @Test - public void testInnerJoinLeftFunction() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinLeftFunction(Map queryContext) throws Exception { testQuery( "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + "FROM foo\n" + "INNER JOIN lookup.lookyloo l ON SUBSTRING(foo.dim2, 1, 1) = l.k\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9393,7 +9922,7 @@ public void testInnerJoinLeftFunction() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1", "dim2", "j0.k", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9405,12 +9934,14 @@ public void testInnerJoinLeftFunction() throws Exception } @Test - public void testInnerJoinRightFunction() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinRightFunction(Map queryContext) throws Exception { testQuery( "SELECT foo.dim1, foo.dim2, l.k, l.v\n" + "FROM foo\n" + "INNER JOIN lookup.lookyloo l ON foo.dim2 = SUBSTRING(l.k, 1, 2)\n", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9425,7 +9956,7 @@ public void testInnerJoinRightFunction() throws Exception ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .columns("k", "v", "v0") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), "j0.", @@ -9435,7 +9966,7 @@ public void testInnerJoinRightFunction() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1", "dim2", "j0.k", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9446,13 +9977,15 @@ public void testInnerJoinRightFunction() throws Exception } @Test - public void testLeftJoinLookupOntoLookupUsingJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testLeftJoinLookupOntoLookupUsingJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT dim2, l1.v, l2.v\n" + "FROM foo\n" + "LEFT JOIN lookup.lookyloo l1 ON foo.dim2 = l1.k\n" + "LEFT JOIN lookup.lookyloo l2 ON l1.k = l2.k", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9472,7 +10005,7 @@ public void testLeftJoinLookupOntoLookupUsingJoinOperator() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("_j0.v", "dim2", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9487,7 +10020,8 @@ public void testLeftJoinLookupOntoLookupUsingJoinOperator() throws Exception } @Test - public void testLeftJoinThreeLookupsUsingJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testLeftJoinThreeLookupsUsingJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT dim1, dim2, l1.v, l2.v, l3.v\n" @@ -9495,6 +10029,7 @@ public void testLeftJoinThreeLookupsUsingJoinOperator() throws Exception + "LEFT JOIN lookup.lookyloo l1 ON foo.dim1 = l1.k\n" + "LEFT JOIN lookup.lookyloo l2 ON foo.dim2 = l2.k\n" + "LEFT JOIN lookup.lookyloo l3 ON l2.k = l3.k", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9520,7 +10055,7 @@ public void testLeftJoinThreeLookupsUsingJoinOperator() throws Exception ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("__j0.v", "_j0.v", "dim1", "dim2", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9535,12 +10070,14 @@ public void testLeftJoinThreeLookupsUsingJoinOperator() throws Exception } @Test - public void testSelectOnLookupUsingLeftJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testSelectOnLookupUsingLeftJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT dim1, lookyloo.*\n" + "FROM foo LEFT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k\n" + "WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9555,7 +10092,7 @@ public void testSelectOnLookupUsingLeftJoinOperator() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .filters(or(not(selector("j0.v", "xxx", null)), selector("j0.v", null, null))) .columns("dim1", "j0.k", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9570,12 +10107,14 @@ public void testSelectOnLookupUsingLeftJoinOperator() throws Exception } @Test - public void testSelectOnLookupUsingRightJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testSelectOnLookupUsingRightJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT dim1, lookyloo.*\n" + "FROM foo RIGHT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k\n" + "WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9590,7 +10129,7 @@ public void testSelectOnLookupUsingRightJoinOperator() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .filters(or(not(selector("j0.v", "xxx", null)), selector("j0.v", null, null))) .columns("dim1", "j0.k", "j0.v") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9603,12 +10142,14 @@ public void testSelectOnLookupUsingRightJoinOperator() throws Exception } @Test - public void testSelectOnLookupUsingFullJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testSelectOnLookupUsingFullJoinOperator(Map queryContext) throws Exception { testQuery( "SELECT dim1, m1, cnt, lookyloo.*\n" + "FROM foo FULL JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k\n" + "WHERE lookyloo.v <> 'xxx' OR lookyloo.v IS NULL", + queryContext, ImmutableList.of( newScanQueryBuilder() .dataSource( @@ -9623,7 +10164,7 @@ public void testSelectOnLookupUsingFullJoinOperator() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .filters(or(not(selector("j0.v", "xxx", null)), selector("j0.v", null, null))) .columns("cnt", "dim1", "j0.k", "j0.v", "m1") - .context(QUERY_CONTEXT_DEFAULT) + .context(queryContext) .build() ), ImmutableList.of( @@ -9681,7 +10222,8 @@ public void testCountDistinctOfLookup() throws Exception } @Test - public void testCountDistinctOfLookupUsingJoinOperator() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testCountDistinctOfLookupUsingJoinOperator(Map queryContext) throws Exception { // Cannot yet vectorize the JOIN operator. cannotVectorize(); @@ -9689,6 +10231,7 @@ public void testCountDistinctOfLookupUsingJoinOperator() throws Exception testQuery( "SELECT COUNT(DISTINCT lookyloo.v)\n" + "FROM foo LEFT JOIN lookup.lookyloo ON foo.dim1 = lookyloo.k", + queryContext, ImmutableList.of( Druids.newTimeseriesQueryBuilder() .dataSource( @@ -9791,7 +10334,7 @@ public void testTimeseries() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.MONTH) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -9887,7 +10430,7 @@ public void testTimeseriesLosAngelesViaQueryContext() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES))) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_LOS_ANGELES) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_LOS_ANGELES, "d0")) .build() ), ImmutableList.of( @@ -9921,7 +10464,7 @@ public void testTimeseriesLosAngelesViaPlannerConfig() throws Exception .intervals(querySegmentSpec(Intervals.of("1999-12-01T00-08:00/2002-01-01T00-08:00"))) .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES))) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -9949,7 +10492,7 @@ public void testTimeseriesUsingTimeFloor() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.MONTH) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10083,7 +10626,7 @@ public void testTimeseriesUsingTimeFloorWithOrigin() throws Exception ) ) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10111,7 +10654,7 @@ public void testTimeseriesLosAngelesUsingTimeFloorConnectionUtc() throws Excepti .intervals(querySegmentSpec(Filtration.eternity())) .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES))) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10142,7 +10685,7 @@ public void testTimeseriesLosAngelesUsingTimeFloorConnectionLosAngeles() throws .intervals(querySegmentSpec(Filtration.eternity())) .granularity(new PeriodGranularity(Period.months(1), null, DateTimes.inferTzFromString(LOS_ANGELES))) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_LOS_ANGELES) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_LOS_ANGELES, "d0")) .build() ), ImmutableList.of( @@ -10175,7 +10718,7 @@ public void testTimeseriesDontSkipEmptyBuckets() throws Exception .intervals(querySegmentSpec(Intervals.of("2000/2000-01-02"))) .granularity(new PeriodGranularity(Period.hours(1), null, DateTimeZone.UTC)) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS) + .context(getTimeseriesContextWithFloorTime(QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS, "d0")) .build() ), ImmutableList.builder() @@ -10223,7 +10766,7 @@ public void testTimeseriesUsingCastAsDate() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(new PeriodGranularity(Period.days(1), null, DateTimeZone.UTC)) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10253,7 +10796,7 @@ public void testTimeseriesUsingFloorPlusCastAsDate() throws Exception .intervals(querySegmentSpec(Filtration.eternity())) .granularity(new PeriodGranularity(Period.months(3), null, DateTimeZone.UTC)) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10283,7 +10826,7 @@ public void testTimeseriesDescending() throws Exception .granularity(Granularities.MONTH) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .descending(true) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10486,7 +11029,7 @@ public void testTimeseriesWithLimitNoTopN() throws Exception .granularity(Granularities.MONTH) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .limit(1) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10513,7 +11056,7 @@ public void testTimeseriesWithLimit() throws Exception .granularity(Granularities.MONTH) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .limit(1) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -10541,7 +11084,7 @@ public void testTimeseriesWithOrderByAndLimit() throws Exception .granularity(Granularities.MONTH) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .limit(1) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -11206,7 +11749,8 @@ public void testGroupingSetsWithOrderByAggregatorWithLimit() throws Exception } @Test - public void testUsingSubqueryAsPartOfAndFilter() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testUsingSubqueryAsPartOfAndFilter(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -11216,6 +11760,7 @@ public void testUsingSubqueryAsPartOfAndFilter() throws Exception + "WHERE dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 <> '')\n" + "AND dim1 <> 'xxx'\n" + "group by dim1, dim2 ORDER BY dim2", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -11255,7 +11800,7 @@ public void testUsingSubqueryAsPartOfAndFilter() throws Exception Integer.MAX_VALUE ) ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -11265,7 +11810,8 @@ public void testUsingSubqueryAsPartOfAndFilter() throws Exception } @Test - public void testUsingSubqueryAsPartOfOrFilter() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testUsingSubqueryAsPartOfOrFilter(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -11274,6 +11820,7 @@ public void testUsingSubqueryAsPartOfOrFilter() throws Exception "SELECT dim1, dim2, COUNT(*) FROM druid.foo\n" + "WHERE dim1 = 'xxx' OR dim2 IN (SELECT dim1 FROM druid.foo WHERE dim1 LIKE '%bc')\n" + "group by dim1, dim2 ORDER BY dim2", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -11307,7 +11854,7 @@ public void testUsingSubqueryAsPartOfOrFilter() throws Exception new DefaultDimensionSpec("v0", "d1", ValueType.LONG) ) ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), "_j0.", @@ -11343,7 +11890,7 @@ public void testUsingSubqueryAsPartOfOrFilter() throws Exception Integer.MAX_VALUE ) ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -11369,80 +11916,78 @@ public void testTimeExtractWithTooFewArguments() throws Exception } @Test - public void testNestedGroupByOnInlineDataSourceWithFilterIsNotSupported() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testNestedGroupByOnInlineDataSourceWithFilter(Map queryContext) throws Exception { - try { - testQuery( - "with abc as" - + "(" - + " SELECT dim1, m2 from druid.foo where \"__time\" >= '2001-01-02'" - + ")" - + ", def as" - + "(" - + " SELECT t1.dim1, SUM(t2.m2) as \"metricSum\" " - + " from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1" - + " where t1.dim1='def'" - + " group by 1" - + ")" - + "SELECT count(*) from def", - ImmutableList.of( - GroupByQuery - .builder() - .setDataSource( - GroupByQuery - .builder() - .setDataSource( - join( - new QueryDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) - .columns("dim1", "m2") - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - new QueryDataSource( - newScanQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) - .columns("dim1", "m2") - .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .context(QUERY_CONTEXT_DEFAULT) - .build() - ), - "j0", - equalsCondition( - DruidExpression.fromColumn("dim1"), - DruidExpression.fromColumn("j0.dim1") - ), - JoinType.INNER - ) - ) - .setGranularity(Granularities.ALL) - .setInterval(querySegmentSpec(Filtration.eternity())) - .build() - ) - .setGranularity(Granularities.ALL) - .setInterval(querySegmentSpec(Filtration.eternity())) - .build() - ), - ImmutableList.of(new Object[] {1}) - ); - Assert.fail("Expected an ISE to be thrown"); - } - catch (RuntimeException e) { - Throwable cause = e.getCause(); - boolean foundISE = false; - while (cause != null) { - if (cause instanceof ISE) { - foundISE = true; - break; - } - cause = cause.getCause(); - } - Assert.assertTrue(foundISE); - } + // Cannot vectorize due to virtual columns. + cannotVectorize(); + + testQuery( + "with abc as" + + "(" + + " SELECT dim1, m2 from druid.foo where \"__time\" >= '2001-01-02'" + + ")" + + ", def as" + + "(" + + " SELECT t1.dim1, SUM(t2.m2) as \"metricSum\" " + + " from abc as t1 inner join abc as t2 on t1.dim1 = t2.dim1" + + " where t1.dim1='def'" + + " group by 1" + + ")" + + "SELECT count(*) from def", + queryContext, + ImmutableList.of( + GroupByQuery + .builder() + .setDataSource( + GroupByQuery + .builder() + .setDataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) + .columns("dim1") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Intervals.of("2001-01-02T00:00:00.000Z/146140482-04-24T15:36:27.903Z"))) + .columns("dim1", "m2") + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim1"), + DruidExpression.fromColumn("j0.dim1") + ), + JoinType.INNER + ) + ) + .setGranularity(Granularities.ALL) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimFilter(selector("dim1", "def", null)) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0") + ) + ) + .setVirtualColumns(expressionVirtualColumn("v0", "'def'", ValueType.STRING)) + .build() + ) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setGranularity(Granularities.ALL) + .setInterval(querySegmentSpec(Filtration.eternity())) + .build() + ), + ImmutableList.of(new Object[] {1L}) + ); } @Test @@ -11680,13 +12225,15 @@ public void testSemiJoinWithOuterTimeExtractAggregateWithOrderBy() throws Except } @Test - public void testInAggregationSubquery() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInAggregationSubquery(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); testQuery( "SELECT DISTINCT __time FROM druid.foo WHERE __time IN (SELECT MAX(__time) FROM druid.foo)", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -11712,7 +12259,7 @@ public void testInAggregationSubquery() throws Exception .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ValueType.LONG))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -11722,13 +12269,15 @@ public void testInAggregationSubquery() throws Exception } @Test - public void testNotInAggregationSubquery() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testNotInAggregationSubquery(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); testQuery( "SELECT DISTINCT __time FROM druid.foo WHERE __time NOT IN (SELECT MAX(__time) FROM druid.foo)", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -11739,14 +12288,13 @@ public void testNotInAggregationSubquery() throws Exception GroupByQuery .builder() .setDataSource( - GroupByQuery - .builder() - .setDataSource(CalciteTests.DATASOURCE1) - .setInterval(querySegmentSpec(Filtration.eternity())) - .setGranularity(Granularities.ALL) - .setAggregatorSpecs(new LongMaxAggregatorFactory("a0", "__time")) - .setContext(QUERY_CONTEXT_DEFAULT) - .build() + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(new LongMaxAggregatorFactory("a0", "__time")) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) @@ -11759,7 +12307,7 @@ public void testNotInAggregationSubquery() throws Exception ) : new CountAggregatorFactory("_a1") ) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), "j0.", @@ -11790,7 +12338,7 @@ public void testNotInAggregationSubquery() throws Exception ) ) .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ValueType.LONG))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -11804,7 +12352,8 @@ public void testNotInAggregationSubquery() throws Exception } @Test - public void testUsingSubqueryWithExtractionFns() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testUsingSubqueryWithExtractionFns(Map queryContext) throws Exception { // Cannot vectorize JOIN operator. cannotVectorize(); @@ -11813,6 +12362,7 @@ public void testUsingSubqueryWithExtractionFns() throws Exception "SELECT dim2, COUNT(*) FROM druid.foo " + "WHERE substring(dim2, 1, 1) IN (SELECT substring(dim1, 1, 1) FROM druid.foo WHERE dim1 <> '')" + "group by dim2", + queryContext, ImmutableList.of( GroupByQuery.builder() .setDataSource( @@ -11849,7 +12399,7 @@ public void testUsingSubqueryWithExtractionFns() throws Exception .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) - .setContext(QUERY_CONTEXT_DEFAULT) + .setContext(queryContext) .build() ), ImmutableList.of( @@ -11860,32 +12410,122 @@ public void testUsingSubqueryWithExtractionFns() throws Exception } @Test - public void testUsingSubqueryWithLimit() throws Exception - { - expectedException.expect(CannotBuildQueryException.class); - expectedException.expectMessage("Subquery could not be converted to GroupBy query"); - - testQuery( - "SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo LIMIT 10 ) tmpA", - ImmutableList.of(), - ImmutableList.of() - ); - } - - @Test - public void testUsingSubqueryWithoutLimit() throws Exception + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinWithIsNullFilter(Map queryContext) throws Exception { testQuery( - "SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo ) tmpA", + "SELECT dim1, l.v from druid.foo f inner join lookup.lookyloo l on f.dim1 = l.k where f.dim2 is null", + queryContext, ImmutableList.of( - Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .granularity(Granularities.ALL) - .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .context(TIMESERIES_CONTEXT_DEFAULT) - .build() - ), + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim1"), + DruidExpression.fromColumn("j0.k") + ), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(selector("dim2", null, null)) + .columns("dim1", "j0.v") + .build() + ), + ImmutableList.of( + new Object[]{"abc", "xabc"} + ) + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + @Ignore // regression test for https://github.com/apache/druid/issues/9924 + public void testInnerJoinOnMultiValueColumn(Map queryContext) throws Exception + { + cannotVectorize(); + testQuery( + "SELECT dim3, l.v, count(*) from druid.foo f inner join lookup.lookyloo l on f.dim3 = l.k " + + "group by 1, 2", + queryContext, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new LookupDataSource("lookyloo"), + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim3"), + DruidExpression.fromColumn("j0.k") + ), + JoinType.INNER + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setDimensions( + dimensions( + new DefaultDimensionSpec("dim3", "d0"), + new DefaultDimensionSpec("j0.v", "d1") + ) + ) + .build() + ), + ImmutableList.of( + new Object[]{"2", "x2", 1L} + ) + ); + } + + @Test + public void testUsingSubqueryWithLimit() throws Exception + { + // Cannot vectorize scan query. + cannotVectorize(); + + testQuery( + "SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo LIMIT 10 ) tmpA", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") + .limit(10) + .context(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .build() + ), + ImmutableList.of( + new Object[]{6L} + ) + ); + } + + @Test + public void testUsingSubqueryWithoutLimit() throws Exception + { + testQuery( + "SELECT COUNT(*) AS cnt FROM ( SELECT * FROM druid.foo ) tmpA", + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(TIMESERIES_CONTEXT_DEFAULT) + .build() + ), ImmutableList.of( new Object[]{6L} ) @@ -12168,7 +12808,7 @@ public void testPostAggWithTimeseries() throws Exception expressionPostAgg("p0", "(\"a0\" + \"a1\")") ) .descending(true) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -12356,7 +12996,7 @@ public void testRequireTimeConditionPositive() throws Exception .intervals(querySegmentSpec(Intervals.of("2000-01-01/2002-01-01"))) .granularity(Granularities.MONTH) .aggregators(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .context(TIMESERIES_CONTEXT_DEFAULT) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_DEFAULT, "d0")) .build() ), ImmutableList.of( @@ -12805,6 +13445,69 @@ public void testNvlColumns() throws Exception ); } + @Test + public void testGroupByWithLiteralInSubqueryGrouping() throws Exception + { + testQuery( + "SELECT \n" + + " t1, t2\n" + + " FROM\n" + + " ( SELECT\n" + + " 'dummy' as t1,\n" + + " CASE\n" + + " WHEN \n" + + " dim4 = 'b'\n" + + " THEN dim4\n" + + " ELSE NULL\n" + + " END AS t2\n" + + " FROM\n" + + " numfoo\n" + + " GROUP BY\n" + + " dim4\n" + + " )\n" + + " GROUP BY\n" + + " t1,t2\n", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE3) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(new DefaultDimensionSpec("dim4", "_d0", ValueType.STRING)) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ) + .setVirtualColumns( + expressionVirtualColumn( + "v0", + "\'dummy\'", + ValueType.STRING + ), + expressionVirtualColumn( + "v1", + "case_searched((\"_d0\" == 'b'),\"_d0\",null)", + ValueType.STRING + ) + ) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v0", "d0", ValueType.STRING), + new DefaultDimensionSpec("v1", "d1", ValueType.STRING) + ) + ) + .setGranularity(Granularities.ALL) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"dummy", NULL_STRING}, + new Object[]{"dummy", "b"} + ) + ); + } + @Test public void testMultiValueStringWorksLikeStringGroupBy() throws Exception { @@ -13884,6 +14587,349 @@ public void testQueryContextOuterLimit() throws Exception ); } + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testCountOnSemiJoinSingleColumn(Map queryContext) throws Exception + { + testQuery( + "SELECT dim1 FROM foo WHERE dim1 IN (SELECT dim1 FROM foo WHERE dim1 = '10.1')\n", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE1), + new QueryDataSource( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setDimFilter( + selector("dim1", "10.1", null) + ) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) + .setContext(queryContext) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("dim1"), DruidExpression.fromColumn("j0.d0")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .columns("v0") + .context(queryContext) + .build() + ), + ImmutableList.of( + new Object[]{"10.1"} + ) + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map queryContext) throws Exception + { + testQuery( + "with abc as\n" + + "(\n" + + " SELECT dim1, \"__time\", m1 from foo WHERE \"dim1\" = '10.1' AND \"__time\" >= '1999'\n" + + ")\n" + + "SELECT t1.dim1, t1.\"__time\" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'\n", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals( + querySegmentSpec( + Intervals.utc( + DateTimes.of("1999-01-01").getMillis(), + JodaUtils.MAX_INSTANT + ) + ) + ) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .columns(ImmutableList.of("__time", "v0")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals( + querySegmentSpec( + Intervals.utc( + DateTimes.of("1999-01-01").getMillis(), + JodaUtils.MAX_INSTANT + ) + ) + ) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .columns(ImmutableList.of("v0")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("v0"), DruidExpression.fromColumn("j0.v0")), + JoinType.LEFT + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .columns("__time", "_v0") + .filters(new SelectorDimFilter("v0", "10.1", null)) + .context(queryContext) + .build() + ), + ImmutableList.of( + new Object[]{"10.1", 946771200000L} + ) + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere(Map queryContext) throws Exception + { + testQuery( + "with abc as\n" + + "(\n" + + " SELECT dim1, \"__time\", m1 from foo WHERE \"dim1\" = '10.1'\n" + + ")\n" + + "SELECT t1.dim1, t1.\"__time\" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'\n", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .columns(ImmutableList.of("__time", "v0")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .columns(ImmutableList.of("dim1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("v0"), DruidExpression.fromColumn("j0.dim1")), + JoinType.LEFT + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .columns("__time", "_v0") + .filters(new SelectorDimFilter("v0", "10.1", null)) + .context(queryContext) + .build() + ), + ImmutableList.of( + new Object[]{"10.1", 946771200000L} + ) + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testLeftJoinOnTwoInlineDataSources(Map queryContext) throws Exception + { + testQuery( + "with abc as\n" + + "(\n" + + " SELECT dim1, \"__time\", m1 from foo WHERE \"dim1\" = '10.1'\n" + + ")\n" + + "SELECT t1.dim1, t1.\"__time\" from abc as t1 LEFT JOIN abc as t2 on t1.dim1 = t2.dim1\n", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .columns(ImmutableList.of("__time", "v0")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .columns(ImmutableList.of("dim1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("v0"), DruidExpression.fromColumn("j0.dim1")), + JoinType.LEFT + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .columns("__time", "_v0") + .context(queryContext) + .build() + ), + ImmutableList.of( + new Object[]{"10.1", 946771200000L} + ) + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinOnTwoInlineDataSourcesWithOuterWhere(Map queryContext) throws Exception + { + testQuery( + "with abc as\n" + + "(\n" + + " SELECT dim1, \"__time\", m1 from foo WHERE \"dim1\" = '10.1'\n" + + ")\n" + + "SELECT t1.dim1, t1.\"__time\" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1 WHERE t1.dim1 = '10.1'\n", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .columns(ImmutableList.of("__time", "v0")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .columns(ImmutableList.of("dim1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("v0"), DruidExpression.fromColumn("j0.dim1")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .columns("__time", "_v0") + .filters(new NotDimFilter(new SelectorDimFilter("v0", null, null))) + .context(queryContext) + .build() + ), + ImmutableList.of( + new Object[]{"10.1", 946771200000L} + ) + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testInnerJoinOnTwoInlineDataSources(Map queryContext) throws Exception + { + testQuery( + "with abc as\n" + + "(\n" + + " SELECT dim1, \"__time\", m1 from foo WHERE \"dim1\" = '10.1'\n" + + ")\n" + + "SELECT t1.dim1, t1.\"__time\" from abc as t1 INNER JOIN abc as t2 on t1.dim1 = t2.dim1\n", + queryContext, + ImmutableList.of( + newScanQueryBuilder() + .dataSource( + join( + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ValueType.STRING)) + .columns(ImmutableList.of("__time", "v0")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + new QueryDataSource( + newScanQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(new SelectorDimFilter("dim1", "10.1", null)) + .columns(ImmutableList.of("dim1")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .context(queryContext) + .build() + ), + "j0.", + equalsCondition(DruidExpression.fromColumn("v0"), DruidExpression.fromColumn("j0.dim1")), + JoinType.INNER + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .virtualColumns(expressionVirtualColumn("_v0", "\'10.1\'", ValueType.STRING)) + .columns("__time", "_v0") + .context(queryContext) + .build() + ), + ImmutableList.of( + new Object[]{"10.1", 946771200000L} + ) + ); + } + + // This query is expected to fail as we do not support join with constant in the on condition + // (see issue https://github.com/apache/druid/issues/9942 for more information) + // TODO: Remove expected Exception when https://github.com/apache/druid/issues/9942 is fixed + @Test(expected = RelOptPlanner.CannotPlanException.class) + @Parameters(source = QueryContextForJoinProvider.class) + public void testJoinOnConstantShouldFail(Map queryContext) throws Exception + { + cannotVectorize(); + + final String query = "SELECT t1.dim1 from foo as t1 LEFT JOIN foo as t2 on t1.dim1 = '10.1'"; + + testQuery( + query, + queryContext, + ImmutableList.of(), + ImmutableList.of() + ); + } + @Test public void testRepeatedIdenticalVirtualExpressionGrouping() throws Exception { @@ -13917,4 +14963,115 @@ public void testRepeatedIdenticalVirtualExpressionGrouping() throws Exception ) ); } + + @Test + public void testValidationErrorNullLiteralIllegal() throws Exception + { + expectedException.expectMessage("Illegal use of 'NULL'"); + + testQuery( + "SELECT REGEXP_LIKE('x', NULL)", + ImmutableList.of(), + ImmutableList.of() + ); + } + + @Test + public void testValidationErrorNonLiteralIllegal() throws Exception + { + expectedException.expectMessage("Argument to function 'REGEXP_LIKE' must be a literal"); + + testQuery( + "SELECT REGEXP_LIKE('x', dim1) FROM foo", + ImmutableList.of(), + ImmutableList.of() + ); + } + + @Test + public void testValidationErrorWrongTypeLiteral() throws Exception + { + expectedException.expectMessage("Cannot apply 'REGEXP_LIKE' to arguments"); + + testQuery( + "SELECT REGEXP_LIKE('x', 1) FROM foo", + ImmutableList.of(), + ImmutableList.of() + ); + } + + @Test + @Parameters(source = QueryContextForJoinProvider.class) + public void testTopNOnStringWithNonSortedOrUniqueDictionary(Map queryContext) throws Exception + { + testQuery( + "SELECT druid.broadcast.dim4, COUNT(*)\n" + + "FROM druid.numfoo\n" + + "INNER JOIN druid.broadcast ON numfoo.dim4 = broadcast.dim4\n" + + "GROUP BY 1 ORDER BY 2 LIMIT 4", + queryContext, + ImmutableList.of( + new TopNQueryBuilder() + .dataSource( + join( + new TableDataSource(CalciteTests.DATASOURCE3), + new GlobalTableDataSource(CalciteTests.BROADCAST_DATASOURCE), + "j0.", + equalsCondition( + DruidExpression.fromColumn("dim4"), + DruidExpression.fromColumn("j0.dim4") + ), + JoinType.INNER + + ) + ) + .intervals(querySegmentSpec(Filtration.eternity())) + .dimension(new DefaultDimensionSpec("j0.dim4", "_d0", ValueType.STRING)) + .threshold(4) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(queryContext) + .metric(new InvertedTopNMetricSpec(new NumericTopNMetricSpec("a0"))) + .build() + ), + ImmutableList.of( + new Object[]{"a", 9L}, + new Object[]{"b", 9L} + ) + ); + } + + /** + * This is a provider of query contexts that should be used by join tests. + * It tests various configs that can be passed to join queries. All the configs provided by this provider should + * have the join query engine return the same results. + */ + public static class QueryContextForJoinProvider + { + @UsedByJUnitParamsRunner + public static Object[] provideQueryContexts() + { + return new Object[] { + // default behavior + QUERY_CONTEXT_DEFAULT, + // filter value re-writes enabled + new ImmutableMap.Builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true) + .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, true) + .build(), + // rewrite values enabled but filter re-writes disabled. + // This should be drive the same behavior as the previous config + new ImmutableMap.Builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(QueryContexts.JOIN_FILTER_REWRITE_VALUE_COLUMN_FILTERS_ENABLE_KEY, true) + .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false) + .build(), + // filter re-writes disabled + new ImmutableMap.Builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(QueryContexts.JOIN_FILTER_REWRITE_ENABLE_KEY, false) + .build(), + }; + } + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index 3136508ee9b7..1d842175ef46 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -30,12 +30,19 @@ import org.apache.calcite.sql.SqlIntervalQualifier; import org.apache.calcite.sql.SqlOperator; import org.apache.calcite.sql.type.SqlTypeName; +import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.Parser; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.segment.RowAdapters; +import org.apache.druid.segment.RowBasedColumnSelectorFactory; +import org.apache.druid.segment.VirtualColumn; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.planner.PlannerContext; +import org.apache.druid.sql.calcite.rel.VirtualColumnRegistry; import org.apache.druid.sql.calcite.table.RowSignatures; import org.apache.druid.sql.calcite.util.CalciteTests; import org.joda.time.DateTime; @@ -46,8 +53,10 @@ import java.math.BigDecimal; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; class ExpressionTestHelper @@ -197,11 +206,11 @@ private static String quoteIfNeeded(@Nullable Object arg) } void testExpression( - SqlTypeName sqlTypeName, - SqlOperator op, - List exprs, - DruidExpression expectedExpression, - Object expectedResult + final SqlTypeName sqlTypeName, + final SqlOperator op, + final List exprs, + final DruidExpression expectedExpression, + final Object expectedResult ) { RelDataType returnType = createSqlType(sqlTypeName); @@ -209,36 +218,79 @@ void testExpression( } void testExpression( - SqlOperator op, - RexNode expr, - DruidExpression expectedExpression, - Object expectedResult + final SqlOperator op, + final RexNode expr, + final DruidExpression expectedExpression, + final Object expectedResult ) { testExpression(op, Collections.singletonList(expr), expectedExpression, expectedResult); } void testExpression( - SqlOperator op, - List exprs, - DruidExpression expectedExpression, - Object expectedResult + final SqlOperator op, + final List exprs, + final DruidExpression expectedExpression, + final Object expectedResult ) { testExpression(rexBuilder.makeCall(op, exprs), expectedExpression, expectedResult); } void testExpression( - RexNode rexNode, - DruidExpression expectedExpression, - Object expectedResult + final RexNode rexNode, + final DruidExpression expectedExpression, + final Object expectedResult ) { DruidExpression expression = Expressions.toDruidExpression(PLANNER_CONTEXT, rowSignature, rexNode); Assert.assertEquals("Expression for: " + rexNode, expectedExpression, expression); - ExprEval result = Parser.parse(expression.getExpression(), PLANNER_CONTEXT.getExprMacroTable()) - .eval(Parser.withMap(bindings)); + ExprEval result = Parser.parse(expression.getExpression(), PLANNER_CONTEXT.getExprMacroTable()) + .eval(Parser.withMap(bindings)); + Assert.assertEquals("Result for: " + rexNode, expectedResult, result.value()); } + + void testFilter( + final SqlOperator op, + final List exprs, + final List expectedVirtualColumns, + final DimFilter expectedFilter, + final boolean expectedResult + ) + { + final RexNode rexNode = rexBuilder.makeCall(op, exprs); + final VirtualColumnRegistry virtualColumnRegistry = VirtualColumnRegistry.create(rowSignature); + + final DimFilter filter = Expressions.toFilter(PLANNER_CONTEXT, rowSignature, virtualColumnRegistry, rexNode); + Assert.assertEquals("Filter for: " + rexNode, expectedFilter, filter); + + final List virtualColumns = + filter.getRequiredColumns() + .stream() + .map(virtualColumnRegistry::getVirtualColumn) + .filter(Objects::nonNull) + .sorted(Comparator.comparing(VirtualColumn::getOutputName)) + .collect(Collectors.toList()); + + Assert.assertEquals( + "Virtual columns for: " + rexNode, + expectedVirtualColumns.stream() + .sorted(Comparator.comparing(VirtualColumn::getOutputName)) + .collect(Collectors.toList()), + virtualColumns + ); + + final ValueMatcher matcher = expectedFilter.toFilter().makeMatcher( + RowBasedColumnSelectorFactory.create( + RowAdapters.standardRow(), + () -> new MapBasedRow(0L, bindings), + rowSignature, + false + ) + ); + + Assert.assertEquals("Result for: " + rexNode, expectedResult, matcher.matches()); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java index d2370d793e9e..9975cffe64cf 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionsTest.java @@ -32,15 +32,19 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.RegexDimExtractionFn; +import org.apache.druid.query.filter.RegexDimFilter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.sql.calcite.expression.builtin.DateTruncOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.LPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.LeftOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ParseLongOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RPadOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RegexpExtractOperatorConversion; +import org.apache.druid.sql.calcite.expression.builtin.RegexpLikeOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RepeatOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.ReverseOperatorConversion; import org.apache.druid.sql.calcite.expression.builtin.RightOperatorConversion; @@ -59,6 +63,7 @@ import org.junit.Test; import java.math.BigDecimal; +import java.util.Collections; import java.util.Map; public class ExpressionsTest extends ExpressionTestBase @@ -72,9 +77,16 @@ public class ExpressionsTest extends ExpressionTestBase .add("y", ValueType.LONG) .add("z", ValueType.FLOAT) .add("s", ValueType.STRING) + .add("nan", ValueType.DOUBLE) + .add("inf", ValueType.DOUBLE) + .add("-inf", ValueType.DOUBLE) + .add("fnan", ValueType.FLOAT) + .add("finf", ValueType.FLOAT) + .add("-finf", ValueType.FLOAT) .add("hexstr", ValueType.STRING) .add("intstr", ValueType.STRING) .add("spacey", ValueType.STRING) + .add("newliney", ValueType.STRING) .add("tstr", ValueType.STRING) .add("dstr", ValueType.STRING) .build(); @@ -86,10 +98,18 @@ public class ExpressionsTest extends ExpressionTestBase .put("x", 2.25) .put("y", 3.0) .put("z", -2.25) + .put("o", 0) + .put("nan", Double.NaN) + .put("inf", Double.POSITIVE_INFINITY) + .put("-inf", Double.NEGATIVE_INFINITY) + .put("fnan", Float.NaN) + .put("finf", Float.POSITIVE_INFINITY) + .put("-finf", Float.NEGATIVE_INFINITY) .put("s", "foo") .put("hexstr", "EF") .put("intstr", "-100") .put("spacey", " hey there ") + .put("newliney", "beep\nboop") .put("tstr", "2000-02-03 04:05:06") .put("dstr", "2000-02-03") .build(); @@ -131,6 +151,50 @@ public void testCharacterLength() @Test public void testRegexpExtract() { + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("x(.)"), + testHelper.makeLiteral(1) + ), + DruidExpression.of( + SimpleExtraction.of("s", new RegexDimExtractionFn("x(.)", 1, true, null)), + "regexp_extract(\"s\",'x(.)',1)" + ), + null + ); + + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("(o)"), + testHelper.makeLiteral(1) + ), + DruidExpression.of( + SimpleExtraction.of("s", new RegexDimExtractionFn("(o)", 1, true, null)), + "regexp_extract(\"s\",'(o)',1)" + ), + + // Column "s" contains an 'o', but not at the beginning; we do match this. + "o" + ); + + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeCall( + SqlStdOperatorTable.CONCAT, + testHelper.makeLiteral("Z"), + testHelper.makeInputRef("s") + ), + testHelper.makeLiteral("Zf(.)") + ), + DruidExpression.fromExpression("regexp_extract(concat('Z',\"s\"),'Zf(.)')"), + "Zfo" + ); + testHelper.testExpression( new RegexpExtractOperatorConversion().calciteOperator(), ImmutableList.of( @@ -157,6 +221,307 @@ public void testRegexpExtract() ), "fo" ); + + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("") + ), + DruidExpression.of( + SimpleExtraction.of("s", new RegexDimExtractionFn("", 0, true, null)), + "regexp_extract(\"s\",'')" + ), + NullHandling.emptyToNullIfNeeded("") + ); + + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("") + ), + DruidExpression.of( + SimpleExtraction.of("s", new RegexDimExtractionFn("", 0, true, null)), + "regexp_extract(\"s\",'')" + ), + NullHandling.emptyToNullIfNeeded("") + ); + + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeNullLiteral(SqlTypeName.VARCHAR), + testHelper.makeLiteral("(.)") + ), + DruidExpression.fromExpression("regexp_extract(null,'(.)')"), + null + ); + + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeNullLiteral(SqlTypeName.VARCHAR), + testHelper.makeLiteral("") + ), + DruidExpression.fromExpression("regexp_extract(null,'')"), + null + ); + + testHelper.testExpression( + new RegexpExtractOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeNullLiteral(SqlTypeName.VARCHAR), + testHelper.makeLiteral("null") + ), + DruidExpression.fromExpression("regexp_extract(null,'null')"), + null + ); + } + + @Test + public void testRegexpLike() + { + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("f.") + ), + DruidExpression.fromExpression("regexp_like(\"s\",'f.')"), + 1L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("o") + ), + DruidExpression.fromExpression("regexp_like(\"s\",'o')"), + + // Column "s" contains an 'o', but not at the beginning; we do match this. + 1L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("x.") + ), + DruidExpression.fromExpression("regexp_like(\"s\",'x.')"), + 0L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("") + ), + DruidExpression.fromExpression("regexp_like(\"s\",'')"), + 1L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeLiteral("beep\nboop"), + testHelper.makeLiteral("^beep$") + ), + DruidExpression.fromExpression("regexp_like('beep\\u000Aboop','^beep$')"), + 0L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeLiteral("beep\nboop"), + testHelper.makeLiteral("^beep\\nboop$") + ), + DruidExpression.fromExpression("regexp_like('beep\\u000Aboop','^beep\\u005Cnboop$')"), + 1L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("newliney"), + testHelper.makeLiteral("^beep$") + ), + DruidExpression.fromExpression("regexp_like(\"newliney\",'^beep$')"), + 0L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("newliney"), + testHelper.makeLiteral("^beep\\nboop$") + ), + DruidExpression.fromExpression("regexp_like(\"newliney\",'^beep\\u005Cnboop$')"), + 1L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("newliney"), + testHelper.makeLiteral("boo") + ), + DruidExpression.fromExpression("regexp_like(\"newliney\",'boo')"), + 1L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("newliney"), + testHelper.makeLiteral("^boo") + ), + DruidExpression.fromExpression("regexp_like(\"newliney\",'^boo')"), + 0L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeCall( + SqlStdOperatorTable.CONCAT, + testHelper.makeLiteral("Z"), + testHelper.makeInputRef("s") + ), + testHelper.makeLiteral("x(.)") + ), + DruidExpression.fromExpression("regexp_like(concat('Z',\"s\"),'x(.)')"), + 0L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeNullLiteral(SqlTypeName.VARCHAR), + testHelper.makeLiteral("(.)") + ), + DruidExpression.fromExpression("regexp_like(null,'(.)')"), + 0L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeNullLiteral(SqlTypeName.VARCHAR), + testHelper.makeLiteral("") + ), + DruidExpression.fromExpression("regexp_like(null,'')"), + + // In SQL-compatible mode, nulls don't match anything. Otherwise, they match like empty strings. + NullHandling.sqlCompatible() ? 0L : 1L + ); + + testHelper.testExpression( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeNullLiteral(SqlTypeName.VARCHAR), + testHelper.makeLiteral("null") + ), + DruidExpression.fromExpression("regexp_like(null,'null')"), + 0L + ); + } + + @Test + public void testRegexpLikeAsFilter() + { + testHelper.testFilter( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("f.") + ), + Collections.emptyList(), + new RegexDimFilter("s", "f.", null), + true + ); + + testHelper.testFilter( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("o") + ), + Collections.emptyList(), + // Column "s" contains an 'o', but not at the beginning, so we don't match + new RegexDimFilter("s", "o", null), + true + ); + + testHelper.testFilter( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("x.") + ), + Collections.emptyList(), + new RegexDimFilter("s", "x.", null), + false + ); + + testHelper.testFilter( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("s"), + testHelper.makeLiteral("") + ), + Collections.emptyList(), + new RegexDimFilter("s", "", null), + true + ); + + testHelper.testFilter( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("newliney"), + testHelper.makeLiteral("^beep$") + ), + Collections.emptyList(), + new RegexDimFilter("newliney", "^beep$", null), + false + ); + + testHelper.testFilter( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeInputRef("newliney"), + testHelper.makeLiteral("^beep\\nboop$") + ), + Collections.emptyList(), + new RegexDimFilter("newliney", "^beep\\nboop$", null), + true + ); + + testHelper.testFilter( + new RegexpLikeOperatorConversion().calciteOperator(), + ImmutableList.of( + testHelper.makeCall( + SqlStdOperatorTable.CONCAT, + testHelper.makeLiteral("Z"), + testHelper.makeInputRef("s") + ), + testHelper.makeLiteral("x(.)") + ), + ImmutableList.of( + new ExpressionVirtualColumn( + "v0", + "concat('Z',\"s\")", + ValueType.STRING, + TestExprMacroTable.INSTANCE + ) + ), + new RegexDimFilter("v0", "x(.)", null), + false + ); } @Test @@ -538,7 +903,7 @@ public void testRoundWithInvalidArgument() expectException( IAE.class, - "The first argument to the function[round] should be integer or double type but get the STRING type" + "The first argument to the function[round] should be integer or double type but got the type: STRING" ); testHelper.testExpression( roundFunction, @@ -555,7 +920,7 @@ public void testRoundWithInvalidSecondArgument() expectException( IAE.class, - "The second argument to the function[round] should be integer type but get the STRING type" + "The second argument to the function[round] should be integer type but got the type: STRING" ); testHelper.testExpression( roundFunction, @@ -568,6 +933,58 @@ public void testRoundWithInvalidSecondArgument() ); } + @Test + public void testRoundWithNanShouldRoundTo0() + { + final SqlFunction roundFunction = new RoundOperatorConversion().calciteOperator(); + + testHelper.testExpression( + roundFunction, + testHelper.makeInputRef("nan"), + DruidExpression.fromExpression("round(\"nan\")"), + 0D + ); + testHelper.testExpression( + roundFunction, + testHelper.makeInputRef("fnan"), + DruidExpression.fromExpression("round(\"fnan\")"), + 0D + ); + } + + @Test + public void testRoundWithInfinityShouldRoundTo0() + { + final SqlFunction roundFunction = new RoundOperatorConversion().calciteOperator(); + + //CHECKSTYLE.OFF: Regexp + testHelper.testExpression( + roundFunction, + testHelper.makeInputRef("inf"), + DruidExpression.fromExpression("round(\"inf\")"), + Double.MAX_VALUE + ); + testHelper.testExpression( + roundFunction, + testHelper.makeInputRef("-inf"), + DruidExpression.fromExpression("round(\"-inf\")"), + -1 * Double.MAX_VALUE + ); + testHelper.testExpression( + roundFunction, + testHelper.makeInputRef("finf"), + DruidExpression.fromExpression("round(\"finf\")"), + Double.MAX_VALUE + ); + testHelper.testExpression( + roundFunction, + testHelper.makeInputRef("-finf"), + DruidExpression.fromExpression("round(\"-finf\")"), + -1 * Double.MAX_VALUE + ); + //CHECKSTYLE.ON: Regexp + } + @Test public void testDateTrunc() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java index fcd96911aecb..58278232ddd4 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/GreatestExpressionTest.java @@ -212,6 +212,23 @@ public void testDecimal() ); } + @Test + public void testDecimalWithNullShouldReturnString() + { + testExpression( + Arrays.asList( + testHelper.makeLiteral(BigDecimal.valueOf(1.2)), + testHelper.getConstantNull(), + testHelper.makeLiteral(BigDecimal.valueOf(3.4)) + ), + buildExpectedExpression( + 1.2, + null, + 3.4 + ), + "3.4" + ); + } @Test public void testTimestamp() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java index 1405eac207c4..6eed91bb241c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/LeastExpressionTest.java @@ -212,6 +212,24 @@ public void testDecimal() ); } + @Test + public void testDecimalWithNullShouldReturnString() + { + testExpression( + Arrays.asList( + testHelper.makeLiteral(BigDecimal.valueOf(1.2)), + testHelper.makeLiteral(BigDecimal.valueOf(3.4)), + testHelper.getConstantNull() + ), + buildExpectedExpression( + 1.2, + 3.4, + null + ), + "1.2" + ); + } + @Test public void testTimestamp() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java index edba60a35c2e..a5e883176039 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidCalciteSchemaModuleTest.java @@ -20,6 +20,7 @@ package org.apache.druid.sql.calcite.schema; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.inject.Guice; import com.google.inject.Injector; @@ -38,7 +39,10 @@ import org.apache.druid.guice.LifecycleModule; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; import org.apache.druid.query.lookup.LookupReferencesManager; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.join.MapJoinableFactory; import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.Escalator; import org.apache.druid.sql.calcite.planner.PlannerConfig; @@ -84,6 +88,8 @@ public class DruidCalciteSchemaModuleTest extends CalciteTestBase private ObjectMapper objectMapper; @Mock private LookupReferencesManager lookupReferencesManager; + @Mock + private SegmentManager segmentManager; private DruidCalciteSchemaModule target; private Injector injector; @@ -99,11 +105,13 @@ public void setUp() binder -> { binder.bind(QueryLifecycleFactory.class).toInstance(queryLifecycleFactory); binder.bind(TimelineServerView.class).toInstance(serverView); + binder.bind(JoinableFactory.class).toInstance(new MapJoinableFactory(ImmutableMap.of())); binder.bind(PlannerConfig.class).toInstance(plannerConfig); binder.bind(ViewManager.class).toInstance(viewManager); binder.bind(Escalator.class).toInstance(escalator); binder.bind(AuthorizerMapper.class).toInstance(authorizerMapper); binder.bind(InventoryView.class).toInstance(serverInventoryView); + binder.bind(SegmentManager.class).toInstance(segmentManager); binder.bind(DruidLeaderClient.class) .annotatedWith(Coordinator.class) .toInstance(coordinatorDruidLeaderClient); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java index 713dfffedf69..0ba26ea9d3ea 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaNoDataInitTest.java @@ -22,7 +22,10 @@ import com.google.common.collect.ImmutableMap; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.segment.join.MapJoinableFactory; +import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.security.NoopEscalator; import org.apache.druid.sql.calcite.planner.PlannerConfig; import org.apache.druid.sql.calcite.util.CalciteTestBase; @@ -30,6 +33,7 @@ import org.apache.druid.sql.calcite.util.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.sql.calcite.util.TestServerInventoryView; import org.apache.druid.sql.calcite.view.NoopViewManager; +import org.easymock.EasyMock; import org.junit.Assert; import org.junit.Test; @@ -50,6 +54,8 @@ public void testInitializationWithNoData() throws Exception conglomerate ), new TestServerInventoryView(Collections.emptyList()), + new SegmentManager(EasyMock.createMock(SegmentLoader.class)), + new MapJoinableFactory(ImmutableMap.of()), PLANNER_CONFIG_DEFAULT, new NoopViewManager(), new NoopEscalator() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java index 6e2f8f62103f..adb3626c54af 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java @@ -22,18 +22,21 @@ 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 org.apache.calcite.jdbc.JavaTypeFactoryImpl; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeField; import org.apache.calcite.schema.Table; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.druid.client.ImmutableDruidServer; -import org.apache.druid.client.TimelineServerView; import org.apache.druid.data.input.InputRow; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.query.DataSource; +import org.apache.druid.query.GlobalTableDataSource; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -41,8 +44,14 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.Joinable; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.join.MapJoinableFactory; +import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.security.NoopEscalator; @@ -58,6 +67,8 @@ import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.partition.LinearShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; +import org.easymock.EasyMock; +import org.joda.time.Period; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; @@ -71,11 +82,22 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; public class DruidSchemaTest extends CalciteTestBase { - private static final PlannerConfig PLANNER_CONFIG_DEFAULT = new PlannerConfig(); + private static final PlannerConfig PLANNER_CONFIG_DEFAULT = new PlannerConfig() + { + @Override + public Period getMetadataRefreshPeriod() + { + return new Period("PT1S"); + } + }; private static final List ROWS1 = ImmutableList.of( CalciteTests.createRow(ImmutableMap.of("t", "2000-01-01", "m1", "1.0", "dim1", "")), @@ -92,7 +114,10 @@ public class DruidSchemaTest extends CalciteTestBase private static QueryRunnerFactoryConglomerate conglomerate; private static Closer resourceCloser; + private TestServerInventoryView serverView; private List druidServers; + private CountDownLatch getDatasourcesLatch = new CountDownLatch(1); + private CountDownLatch buildTableLatch = new CountDownLatch(1); @BeforeClass public static void setUpClass() @@ -112,10 +137,16 @@ public static void tearDownClass() throws IOException private SpecificSegmentsQuerySegmentWalker walker = null; private DruidSchema schema = null; + private SegmentManager segmentManager; + private Set segmentDataSourceNames; + private Set joinableDataSourceNames; @Before public void setUp() throws Exception { + segmentDataSourceNames = Sets.newConcurrentHashSet(); + joinableDataSourceNames = Sets.newConcurrentHashSet(); + final File tmpDir = temporaryFolder.newFolder(); final QueryableIndex index1 = IndexBuilder.create() .tmpDir(new File(tmpDir, "1")) @@ -145,6 +176,16 @@ public void setUp() throws Exception .rows(ROWS2) .buildMMappedIndex(); + segmentManager = new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + { + @Override + public Set getDataSourceNames() + { + getDatasourcesLatch.countDown(); + return segmentDataSourceNames; + } + }; + walker = new SpecificSegmentsQuerySegmentWalker(conglomerate).add( DataSegment.builder() .dataSource(CalciteTests.DATASOURCE1) @@ -187,16 +228,46 @@ public void setUp() throws Exception PruneSpecsHolder.DEFAULT ); final List realtimeSegments = ImmutableList.of(segment1); - final TimelineServerView serverView = new TestServerInventoryView(walker.getSegments(), realtimeSegments); + serverView = new TestServerInventoryView(walker.getSegments(), realtimeSegments); druidServers = serverView.getDruidServers(); + final JoinableFactory globalTableJoinable = new JoinableFactory() + { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return dataSource instanceof GlobalTableDataSource && + joinableDataSourceNames.contains(((GlobalTableDataSource) dataSource).getName()); + } + + @Override + public Optional build( + DataSource dataSource, + JoinConditionAnalysis condition + ) + { + return Optional.empty(); + } + }; + schema = new DruidSchema( CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), serverView, + segmentManager, + new MapJoinableFactory(ImmutableMap.of(GlobalTableDataSource.class, globalTableJoinable)), PLANNER_CONFIG_DEFAULT, new NoopViewManager(), new NoopEscalator() - ); + ) + { + @Override + protected DruidTable buildDruidTable(String dataSource) + { + DruidTable table = super.buildDruidTable(dataSource); + buildTableLatch.countDown(); + return table; + } + }; schema.start(); schema.awaitInitialization(); @@ -418,4 +489,136 @@ public void testAvailableSegmentMetadataIsRealtime() Assert.assertEquals(0L, currentMetadata.isRealtime()); } + @Test + public void testLocalSegmentCacheSetsDataSourceAsGlobalAndJoinable() throws InterruptedException + { + DruidTable fooTable = (DruidTable) schema.getTableMap().get("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.getDataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.getDataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isJoinable()); + Assert.assertFalse(fooTable.isBroadcast()); + + buildTableLatch.await(1, TimeUnit.SECONDS); + + final DataSegment someNewBrokerSegment = new DataSegment( + "foo", + Intervals.of("2012/2013"), + "version1", + null, + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("met1", "met2"), + new NumberedShardSpec(2, 3), + null, + 1, + 100L, + PruneSpecsHolder.DEFAULT + ); + segmentDataSourceNames.add("foo"); + joinableDataSourceNames.add("foo"); + serverView.addSegment(someNewBrokerSegment, ServerType.BROKER); + + // wait for build twice + buildTableLatch = new CountDownLatch(2); + buildTableLatch.await(1, TimeUnit.SECONDS); + + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + getDatasourcesLatch = new CountDownLatch(1); + getDatasourcesLatch.await(1, TimeUnit.SECONDS); + + fooTable = (DruidTable) schema.getTableMap().get("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.getDataSource() instanceof TableDataSource); + Assert.assertTrue(fooTable.getDataSource() instanceof GlobalTableDataSource); + Assert.assertTrue(fooTable.isJoinable()); + Assert.assertTrue(fooTable.isBroadcast()); + + // now remove it + joinableDataSourceNames.remove("foo"); + segmentDataSourceNames.remove("foo"); + serverView.removeSegment(someNewBrokerSegment, ServerType.BROKER); + + // wait for build + buildTableLatch.await(1, TimeUnit.SECONDS); + buildTableLatch = new CountDownLatch(1); + buildTableLatch.await(1, TimeUnit.SECONDS); + + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + getDatasourcesLatch = new CountDownLatch(1); + getDatasourcesLatch.await(1, TimeUnit.SECONDS); + + fooTable = (DruidTable) schema.getTableMap().get("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.getDataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.getDataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isJoinable()); + Assert.assertFalse(fooTable.isBroadcast()); + } + + @Test + public void testLocalSegmentCacheSetsDataSourceAsBroadcastButNotJoinable() throws InterruptedException + { + DruidTable fooTable = (DruidTable) schema.getTableMap().get("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.getDataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.getDataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isJoinable()); + Assert.assertFalse(fooTable.isBroadcast()); + + // wait for build twice + buildTableLatch.await(1, TimeUnit.SECONDS); + + final DataSegment someNewBrokerSegment = new DataSegment( + "foo", + Intervals.of("2012/2013"), + "version1", + null, + ImmutableList.of("dim1", "dim2"), + ImmutableList.of("met1", "met2"), + new NumberedShardSpec(2, 3), + null, + 1, + 100L, + PruneSpecsHolder.DEFAULT + ); + segmentDataSourceNames.add("foo"); + serverView.addSegment(someNewBrokerSegment, ServerType.BROKER); + + buildTableLatch = new CountDownLatch(2); + buildTableLatch.await(1, TimeUnit.SECONDS); + + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + getDatasourcesLatch = new CountDownLatch(1); + getDatasourcesLatch.await(1, TimeUnit.SECONDS); + + fooTable = (DruidTable) schema.getTableMap().get("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.getDataSource() instanceof TableDataSource); + // should not be a GlobalTableDataSource for now, because isGlobal is couple with joinability. idealy this will be + // changed in the future and we should expect + Assert.assertFalse(fooTable.getDataSource() instanceof GlobalTableDataSource); + Assert.assertTrue(fooTable.isBroadcast()); + Assert.assertFalse(fooTable.isJoinable()); + + + // now remove it + segmentDataSourceNames.remove("foo"); + serverView.removeSegment(someNewBrokerSegment, ServerType.BROKER); + + // wait for build + buildTableLatch.await(1, TimeUnit.SECONDS); + buildTableLatch = new CountDownLatch(1); + buildTableLatch.await(1, TimeUnit.SECONDS); + + // wait for get again, just to make sure table has been updated (latch counts down just before tables are updated) + getDatasourcesLatch = new CountDownLatch(1); + getDatasourcesLatch.await(1, TimeUnit.SECONDS); + + fooTable = (DruidTable) schema.getTableMap().get("foo"); + Assert.assertNotNull(fooTable); + Assert.assertTrue(fooTable.getDataSource() instanceof TableDataSource); + Assert.assertFalse(fooTable.getDataSource() instanceof GlobalTableDataSource); + Assert.assertFalse(fooTable.isBroadcast()); + Assert.assertFalse(fooTable.isJoinable()); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 80bf83030f35..ace7acaa17f8 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -67,9 +67,12 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.join.MapJoinableFactory; +import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; @@ -239,6 +242,8 @@ public Authorizer getAuthorizer(String name) druidSchema = new DruidSchema( CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), new TestServerInventoryView(walker.getSegments(), realtimeSegments), + new SegmentManager(EasyMock.createMock(SegmentLoader.class)), + new MapJoinableFactory(ImmutableMap.of()), PLANNER_CONFIG_DEFAULT, new NoopViewManager(), new NoopEscalator() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 08742812f89b..dfee466bcef3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -50,13 +50,17 @@ import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.ExpressionModule; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.HttpResponseHandler; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.DataSource; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; +import org.apache.druid.query.GlobalTableDataSource; +import org.apache.druid.query.InlineDataSource; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QuerySegmentWalker; @@ -72,12 +76,21 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.incremental.IncrementalIndexSchema; +import org.apache.druid.segment.join.JoinConditionAnalysis; +import org.apache.druid.segment.join.Joinable; +import org.apache.druid.segment.join.JoinableFactory; +import org.apache.druid.segment.join.table.IndexedTableJoinable; +import org.apache.druid.segment.join.table.RowBasedIndexedTable; +import org.apache.druid.segment.loading.SegmentLoader; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryLifecycleFactory; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryStackTests; +import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; import org.apache.druid.server.log.NoopRequestLogger; import org.apache.druid.server.security.Access; @@ -107,6 +120,7 @@ import org.apache.druid.sql.calcite.view.ViewManager; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.LinearShardSpec; +import org.easymock.EasyMock; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.chrono.ISOChronology; @@ -121,9 +135,11 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.Executor; import java.util.function.BooleanSupplier; +import java.util.stream.Collectors; /** * Utility functions for Calcite tests. @@ -135,7 +151,11 @@ public class CalciteTests public static final String DATASOURCE3 = "numfoo"; public static final String DATASOURCE4 = "foo4"; public static final String DATASOURCE5 = "lotsocolumns"; + public static final String BROADCAST_DATASOURCE = "broadcast"; public static final String FORBIDDEN_DATASOURCE = "forbiddenDatasource"; + public static final String SOME_DATASOURCE = "some_datasource"; + public static final String SOME_DATSOURCE_ESCAPED = "some\\_datasource"; + public static final String SOMEXDATASOURCE = "somexdatasource"; public static final String DRUID_SCHEMA_NAME = "druid"; public static final String INFORMATION_SCHEMA_NAME = "INFORMATION_SCHEMA"; public static final String SYSTEM_SCHEMA_NAME = "sys"; @@ -206,7 +226,7 @@ public AuthenticationResult createEscalatedAuthenticationResult() private static final String TIMESTAMP_COLUMN = "t"; - private static final Injector INJECTOR = Guice.createInjector( + public static final Injector INJECTOR = Guice.createInjector( binder -> { binder.bind(Key.get(ObjectMapper.class, Json.class)).toInstance(TestHelper.makeJsonMapper()); @@ -290,6 +310,16 @@ public AuthenticationResult createEscalatedAuthenticationResult() .withRollup(false) .build(); + private static final IncrementalIndexSchema INDEX_SCHEMA_WITH_X_COLUMNS = new IncrementalIndexSchema.Builder() + .withMetrics( + new CountAggregatorFactory("cnt_x"), + new FloatSumAggregatorFactory("m1_x", "m1_x"), + new DoubleSumAggregatorFactory("m2_x", "m2_x"), + new HyperUniquesAggregatorFactory("unique_dim1_x", "dim1_x") + ) + .withRollup(false) + .build(); + private static final IncrementalIndexSchema INDEX_SCHEMA_NUMERIC_DIMS = new IncrementalIndexSchema.Builder() .withMetrics( new CountAggregatorFactory("cnt"), @@ -309,189 +339,240 @@ public AuthenticationResult createEscalatedAuthenticationResult() .withRollup(false) .build(); - public static final List ROWS1 = ImmutableList.of( + public static final List> RAW_ROWS1 = ImmutableList.of( + ImmutableMap.builder() + .put("t", "2000-01-01") + .put("m1", "1.0") + .put("m2", "1.0") + .put("dim1", "") + .put("dim2", ImmutableList.of("a")) + .put("dim3", ImmutableList.of("a", "b")) + .build(), + ImmutableMap.builder() + .put("t", "2000-01-02") + .put("m1", "2.0") + .put("m2", "2.0") + .put("dim1", "10.1") + .put("dim2", ImmutableList.of()) + .put("dim3", ImmutableList.of("b", "c")) + .build(), + ImmutableMap.builder() + .put("t", "2000-01-03") + .put("m1", "3.0") + .put("m2", "3.0") + .put("dim1", "2") + .put("dim2", ImmutableList.of("")) + .put("dim3", ImmutableList.of("d")) + .build(), + ImmutableMap.builder() + .put("t", "2001-01-01") + .put("m1", "4.0") + .put("m2", "4.0") + .put("dim1", "1") + .put("dim2", ImmutableList.of("a")) + .put("dim3", ImmutableList.of("")) + .build(), + ImmutableMap.builder() + .put("t", "2001-01-02") + .put("m1", "5.0") + .put("m2", "5.0") + .put("dim1", "def") + .put("dim2", ImmutableList.of("abc")) + .put("dim3", ImmutableList.of()) + .build(), + ImmutableMap.builder() + .put("t", "2001-01-03") + .put("m1", "6.0") + .put("m2", "6.0") + .put("dim1", "abc") + .build() + ); + + public static final List RAW_ROWS1_X = ImmutableList.of( createRow( ImmutableMap.builder() .put("t", "2000-01-01") - .put("m1", "1.0") - .put("m2", "1.0") - .put("dim1", "") - .put("dim2", ImmutableList.of("a")) - .put("dim3", ImmutableList.of("a", "b")) + .put("m1_x", "1.0") + .put("m2_x", "1.0") + .put("dim1_x", "") + .put("dim2_x", ImmutableList.of("a")) + .put("dim3_x", ImmutableList.of("a", "b")) .build() ), createRow( ImmutableMap.builder() .put("t", "2000-01-02") - .put("m1", "2.0") - .put("m2", "2.0") - .put("dim1", "10.1") - .put("dim2", ImmutableList.of()) - .put("dim3", ImmutableList.of("b", "c")) + .put("m1_x", "2.0") + .put("m2_x", "2.0") + .put("dim1_x", "10.1") + .put("dim2_x", ImmutableList.of()) + .put("dim3_x", ImmutableList.of("b", "c")) .build() ), createRow( ImmutableMap.builder() .put("t", "2000-01-03") - .put("m1", "3.0") - .put("m2", "3.0") - .put("dim1", "2") - .put("dim2", ImmutableList.of("")) - .put("dim3", ImmutableList.of("d")) + .put("m1_x", "3.0") + .put("m2_x", "3.0") + .put("dim1_x", "2") + .put("dim2_x", ImmutableList.of("")) + .put("dim3_x", ImmutableList.of("d")) .build() ), createRow( ImmutableMap.builder() .put("t", "2001-01-01") - .put("m1", "4.0") - .put("m2", "4.0") - .put("dim1", "1") - .put("dim2", ImmutableList.of("a")) - .put("dim3", ImmutableList.of("")) + .put("m1_x", "4.0") + .put("m2_x", "4.0") + .put("dim1_x", "1") + .put("dim2_x", ImmutableList.of("a")) + .put("dim3_x", ImmutableList.of("")) .build() ), createRow( ImmutableMap.builder() .put("t", "2001-01-02") - .put("m1", "5.0") - .put("m2", "5.0") - .put("dim1", "def") - .put("dim2", ImmutableList.of("abc")) - .put("dim3", ImmutableList.of()) + .put("m1_x", "5.0") + .put("m2_x", "5.0") + .put("dim1_x", "def") + .put("dim2_x", ImmutableList.of("abc")) + .put("dim3_x", ImmutableList.of()) .build() ), createRow( ImmutableMap.builder() .put("t", "2001-01-03") - .put("m1", "6.0") - .put("m2", "6.0") - .put("dim1", "abc") + .put("m1_x", "6.0") + .put("m2_x", "6.0") + .put("dim1_x", "abc") .build() ) ); - public static final List ROWS1_WITH_NUMERIC_DIMS = ImmutableList.of( - createRow( - ImmutableMap.builder() - .put("t", "2000-01-01") - .put("m1", "1.0") - .put("m2", "1.0") - .put("d1", 1.0) - .put("f1", 1.0f) - .put("l1", 7L) - .put("dim1", "") - .put("dim2", ImmutableList.of("a")) - .put("dim3", ImmutableList.of("a", "b")) - .put("dim4", "a") - .put("dim5", "aa") - .build(), - PARSER_NUMERIC_DIMS - ), - createRow( - ImmutableMap.builder() - .put("t", "2000-01-02") - .put("m1", "2.0") - .put("m2", "2.0") - .put("d1", 1.7) - .put("d2", 1.7) - .put("f1", 0.1f) - .put("f2", 0.1f) - .put("l1", 325323L) - .put("l2", 325323L) - .put("dim1", "10.1") - .put("dim2", ImmutableList.of()) - .put("dim3", ImmutableList.of("b", "c")) - .put("dim4", "a") - .put("dim5", "ab") - .build(), - PARSER_NUMERIC_DIMS - ), - createRow( - ImmutableMap.builder() - .put("t", "2000-01-03") - .put("m1", "3.0") - .put("m2", "3.0") - .put("d1", 0.0) - .put("d2", 0.0) - .put("f1", 0.0) - .put("f2", 0.0) - .put("l1", 0) - .put("l2", 0) - .put("dim1", "2") - .put("dim2", ImmutableList.of("")) - .put("dim3", ImmutableList.of("d")) - .put("dim4", "a") - .put("dim5", "ba") - .build(), - PARSER_NUMERIC_DIMS - ), - createRow( - ImmutableMap.builder() - .put("t", "2001-01-01") - .put("m1", "4.0") - .put("m2", "4.0") - .put("dim1", "1") - .put("dim2", ImmutableList.of("a")) - .put("dim3", ImmutableList.of("")) - .put("dim4", "b") - .put("dim5", "ad") - .build(), - PARSER_NUMERIC_DIMS - ), - createRow( - ImmutableMap.builder() - .put("t", "2001-01-02") - .put("m1", "5.0") - .put("m2", "5.0") - .put("dim1", "def") - .put("dim2", ImmutableList.of("abc")) - .put("dim3", ImmutableList.of()) - .put("dim4", "b") - .put("dim5", "aa") - .build(), - PARSER_NUMERIC_DIMS - ), - createRow( - ImmutableMap.builder() - .put("t", "2001-01-03") - .put("m1", "6.0") - .put("m2", "6.0") - .put("dim1", "abc") - .put("dim4", "b") - .put("dim5", "ab") - .build(), - PARSER_NUMERIC_DIMS - ) + public static final List ROWS1 = + RAW_ROWS1.stream().map(CalciteTests::createRow).collect(Collectors.toList()); + + public static final List> RAW_ROWS1_WITH_NUMERIC_DIMS = ImmutableList.of( + ImmutableMap.builder() + .put("t", "2000-01-01") + .put("m1", "1.0") + .put("m2", "1.0") + .put("d1", 1.0) + .put("f1", 1.0f) + .put("l1", 7L) + .put("dim1", "") + .put("dim2", ImmutableList.of("a")) + .put("dim3", ImmutableList.of("a", "b")) + .put("dim4", "a") + .put("dim5", "aa") + .build(), + ImmutableMap.builder() + .put("t", "2000-01-02") + .put("m1", "2.0") + .put("m2", "2.0") + .put("d1", 1.7) + .put("d2", 1.7) + .put("f1", 0.1f) + .put("f2", 0.1f) + .put("l1", 325323L) + .put("l2", 325323L) + .put("dim1", "10.1") + .put("dim2", ImmutableList.of()) + .put("dim3", ImmutableList.of("b", "c")) + .put("dim4", "a") + .put("dim5", "ab") + .build(), + ImmutableMap.builder() + .put("t", "2000-01-03") + .put("m1", "3.0") + .put("m2", "3.0") + .put("d1", 0.0) + .put("d2", 0.0) + .put("f1", 0.0) + .put("f2", 0.0) + .put("l1", 0) + .put("l2", 0) + .put("dim1", "2") + .put("dim2", ImmutableList.of("")) + .put("dim3", ImmutableList.of("d")) + .put("dim4", "a") + .put("dim5", "ba") + .build(), + ImmutableMap.builder() + .put("t", "2001-01-01") + .put("m1", "4.0") + .put("m2", "4.0") + .put("dim1", "1") + .put("dim2", ImmutableList.of("a")) + .put("dim3", ImmutableList.of("")) + .put("dim4", "b") + .put("dim5", "ad") + .build(), + ImmutableMap.builder() + .put("t", "2001-01-02") + .put("m1", "5.0") + .put("m2", "5.0") + .put("dim1", "def") + .put("dim2", ImmutableList.of("abc")) + .put("dim3", ImmutableList.of()) + .put("dim4", "b") + .put("dim5", "aa") + .build(), + ImmutableMap.builder() + .put("t", "2001-01-03") + .put("m1", "6.0") + .put("m2", "6.0") + .put("dim1", "abc") + .put("dim4", "b") + .put("dim5", "ab") + .build() ); - - public static final List ROWS2 = ImmutableList.of( - createRow("2000-01-01", "דרואיד", "he", 1.0), - createRow("2000-01-01", "druid", "en", 1.0), - createRow("2000-01-01", "друид", "ru", 1.0) + public static final List ROWS1_WITH_NUMERIC_DIMS = + RAW_ROWS1_WITH_NUMERIC_DIMS.stream().map(raw -> createRow(raw, PARSER_NUMERIC_DIMS)).collect(Collectors.toList()); + + public static final List> RAW_ROWS2 = ImmutableList.of( + ImmutableMap.builder() + .put("t", "2000-01-01") + .put("dim1", "דרואיד") + .put("dim2", "he") + .put("m1", 1.0) + .build(), + ImmutableMap.builder() + .put("t", "2000-01-01") + .put("dim1", "druid") + .put("dim2", "en") + .put("m1", 1.0) + .build(), + ImmutableMap.builder() + .put("t", "2000-01-01") + .put("dim1", "друид") + .put("dim2", "ru") + .put("m1", 1.0) + .build() ); - - public static final List ROWS1_WITH_FULL_TIMESTAMP = ImmutableList.of( - createRow( - ImmutableMap.builder() - .put("t", "2000-01-01T10:51:45.695Z") - .put("m1", "1.0") - .put("m2", "1.0") - .put("dim1", "") - .put("dim2", ImmutableList.of("a")) - .put("dim3", ImmutableList.of("a", "b")) - .build() - ), - createRow( - ImmutableMap.builder() - .put("t", "2000-01-18T10:51:45.695Z") - .put("m1", "2.0") - .put("m2", "2.0") - .put("dim1", "10.1") - .put("dim2", ImmutableList.of()) - .put("dim3", ImmutableList.of("b", "c")) - .build() - ) + public static final List ROWS2 = + RAW_ROWS2.stream().map(CalciteTests::createRow).collect(Collectors.toList()); + + public static final List> RAW_ROWS1_WITH_FULL_TIMESTAMP = ImmutableList.of( + ImmutableMap.builder() + .put("t", "2000-01-01T10:51:45.695Z") + .put("m1", "1.0") + .put("m2", "1.0") + .put("dim1", "") + .put("dim2", ImmutableList.of("a")) + .put("dim3", ImmutableList.of("a", "b")) + .build(), + ImmutableMap.builder() + .put("t", "2000-01-18T10:51:45.695Z") + .put("m1", "2.0") + .put("m2", "2.0") + .put("dim1", "10.1") + .put("dim2", ImmutableList.of()) + .put("dim3", ImmutableList.of("b", "c")) + .build() ); + public static final List ROWS1_WITH_FULL_TIMESTAMP = + RAW_ROWS1_WITH_FULL_TIMESTAMP.stream().map(CalciteTests::createRow).collect(Collectors.toList()); public static final List FORBIDDEN_ROWS = ImmutableList.of( @@ -536,6 +617,68 @@ public AuthenticationResult createEscalatedAuthenticationResult() ) ); + private static final InlineDataSource JOINABLE_BACKING_DATA = InlineDataSource.fromIterable( + RAW_ROWS1_WITH_NUMERIC_DIMS.stream().map(x -> new Object[]{ + x.get("dim1"), + x.get("dim2"), + x.get("dim3"), + x.get("dim4"), + x.get("dim5"), + x.get("d1"), + x.get("d2"), + x.get("f1"), + x.get("f2"), + x.get("l1"), + x.get("l2") + }).collect(Collectors.toList()), + RowSignature.builder() + .add("dim1", ValueType.STRING) + .add("dim2", ValueType.STRING) + .add("dim3", ValueType.STRING) + .add("dim4", ValueType.STRING) + .add("dim5", ValueType.STRING) + .add("d1", ValueType.DOUBLE) + .add("d2", ValueType.DOUBLE) + .add("f1", ValueType.FLOAT) + .add("f2", ValueType.FLOAT) + .add("l1", ValueType.LONG) + .add("l2", ValueType.LONG) + .build() + ); + + private static final Set KEY_COLUMNS = ImmutableSet.of("dim4"); + + private static final RowBasedIndexedTable JOINABLE_TABLE = new RowBasedIndexedTable( + JOINABLE_BACKING_DATA.getRowsAsList(), + JOINABLE_BACKING_DATA.rowAdapter(), + JOINABLE_BACKING_DATA.getRowSignature(), + KEY_COLUMNS, + DateTimes.nowUtc().toString() + ); + + public static GlobalTableDataSource CUSTOM_TABLE = new GlobalTableDataSource(BROADCAST_DATASOURCE); + + public static JoinableFactory CUSTOM_ROW_TABLE_JOINABLE = new JoinableFactory() + { + @Override + public boolean isDirectlyJoinable(DataSource dataSource) + { + return CUSTOM_TABLE.equals(dataSource); + } + + @Override + public Optional build( + DataSource dataSource, + JoinConditionAnalysis condition + ) + { + if (dataSource instanceof GlobalTableDataSource) { + return Optional.of(new IndexedTableJoinable(JOINABLE_TABLE)); + } + return Optional.empty(); + } + }; + private CalciteTests() { // No instantiation. @@ -580,12 +723,28 @@ public static ObjectMapper getJsonMapper() return INJECTOR.getInstance(Key.get(ObjectMapper.class, Json.class)); } + public static JoinableFactory createDefaultJoinableFactory() + { + return QueryStackTests.makeJoinableFactoryFromDefault( + INJECTOR.getInstance(LookupExtractorFactoryContainerProvider.class), + ImmutableMap.of( + GlobalTableDataSource.class, + CUSTOM_ROW_TABLE_JOINABLE + ) + ); + } + public static SpecificSegmentsQuerySegmentWalker createMockWalker( final QueryRunnerFactoryConglomerate conglomerate, final File tmpDir ) { - return createMockWalker(conglomerate, tmpDir, QueryStackTests.DEFAULT_NOOP_SCHEDULER); + return createMockWalker( + conglomerate, + tmpDir, + QueryStackTests.DEFAULT_NOOP_SCHEDULER, + createDefaultJoinableFactory() + ); } public static SpecificSegmentsQuerySegmentWalker createMockWalker( @@ -593,6 +752,16 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( final File tmpDir, final QueryScheduler scheduler ) + { + return createMockWalker(conglomerate, tmpDir, scheduler, null); + } + + public static SpecificSegmentsQuerySegmentWalker createMockWalker( + final QueryRunnerFactoryConglomerate conglomerate, + final File tmpDir, + final QueryScheduler scheduler, + final JoinableFactory joinableFactory + ) { final QueryableIndex index1 = IndexBuilder .create() @@ -642,11 +811,27 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .rows(ROWS_LOTS_OF_COLUMNS) .buildMMappedIndex(); + final QueryableIndex someDatasourceIndex = IndexBuilder + .create() + .tmpDir(new File(tmpDir, "6")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(INDEX_SCHEMA) + .rows(ROWS1) + .buildMMappedIndex(); + + final QueryableIndex someXDatasourceIndex = IndexBuilder + .create() + .tmpDir(new File(tmpDir, "7")) + .segmentWriteOutMediumFactory(OffHeapMemorySegmentWriteOutMediumFactory.instance()) + .schema(INDEX_SCHEMA_WITH_X_COLUMNS) + .rows(RAW_ROWS1_X) + .buildMMappedIndex(); + return new SpecificSegmentsQuerySegmentWalker( conglomerate, INJECTOR.getInstance(LookupExtractorFactoryContainerProvider.class), - null, + joinableFactory, scheduler ).add( DataSegment.builder() @@ -702,6 +887,33 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .size(0) .build(), indexLotsOfColumns + ).add( + DataSegment.builder() + .dataSource(SOME_DATASOURCE) + .interval(indexLotsOfColumns.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(), + someDatasourceIndex + ).add( + DataSegment.builder() + .dataSource(SOMEXDATASOURCE) + .interval(indexLotsOfColumns.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(), + someXDatasourceIndex + ).add( + DataSegment.builder() + .dataSource(BROADCAST_DATASOURCE) + .interval(indexNumericDims.getDataInterval()) + .version("1") + .shardSpec(new LinearShardSpec(0)) + .size(0) + .build(), + indexNumericDims ); } @@ -876,6 +1088,15 @@ private static DruidSchema createMockSchema( final DruidSchema schema = new DruidSchema( CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate), new TestServerInventoryView(walker.getSegments()), + new SegmentManager(EasyMock.createMock(SegmentLoader.class)) + { + @Override + public Set getDataSourceNames() + { + return ImmutableSet.of(BROADCAST_DATASOURCE); + } + }, + createDefaultJoinableFactory(), plannerConfig, viewManager, TEST_AUTHENTICATOR_ESCALATOR diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 1da0fbabf9c9..7fde642ca8c0 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -40,10 +40,7 @@ import org.apache.druid.segment.ReferenceCountingSegment; import org.apache.druid.segment.Segment; import org.apache.druid.segment.SegmentWrangler; -import org.apache.druid.segment.join.InlineJoinableFactory; import org.apache.druid.segment.join.JoinableFactory; -import org.apache.druid.segment.join.LookupJoinableFactory; -import org.apache.druid.segment.join.MapJoinableFactoryTest; import org.apache.druid.server.ClientQuerySegmentWalker; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.QueryStackTests; @@ -91,12 +88,7 @@ public SpecificSegmentsQuerySegmentWalker( final JoinableFactory joinableFactoryToUse; if (joinableFactory == null) { - joinableFactoryToUse = MapJoinableFactoryTest.fromMap( - ImmutableMap., JoinableFactory>builder() - .put(InlineDataSource.class, new InlineJoinableFactory()) - .put(LookupDataSource.class, new LookupJoinableFactory(lookupProvider)) - .build() - ); + joinableFactoryToUse = QueryStackTests.makeJoinableFactoryForLookup(lookupProvider); } else { joinableFactoryToUse = joinableFactory; } @@ -120,6 +112,7 @@ public SpecificSegmentsQuerySegmentWalker( scheduler ), conglomerate, + joinableFactoryToUse, new ServerConfig() ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java index a8e498beb1f0..170d205f449d 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/TestServerInventoryView.java @@ -26,6 +26,7 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.TimelineServerView; import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -63,18 +64,31 @@ public class TestServerInventoryView implements TimelineServerView "dummy", 0 ); - private final List segments; + private static final DruidServerMetadata DUMMY_BROKER = new DruidServerMetadata( + "dummy3", + "dummy3", + null, + 0, + ServerType.BROKER, + "dummy", + 0 + ); + private List segments = new ArrayList<>(); private List realtimeSegments = new ArrayList<>(); + private List brokerSegments = new ArrayList<>(); + + private List> segmentCallbackExecs = new ArrayList<>(); + private List> timelineCallbackExecs = new ArrayList<>(); public TestServerInventoryView(List segments) { - this.segments = ImmutableList.copyOf(segments); + this.segments.addAll(segments); } public TestServerInventoryView(List segments, List realtimeSegments) { - this.segments = ImmutableList.copyOf(segments); - this.realtimeSegments = ImmutableList.copyOf(realtimeSegments); + this.segments.addAll(segments); + this.realtimeSegments.addAll(realtimeSegments); } @Override @@ -87,6 +101,7 @@ public Optional> getTimeline(Da @Override public List getDruidServers() { + // do not return broker on purpose to mimic behavior of BrokerServerView final ImmutableDruidDataSource dataSource = new ImmutableDruidDataSource("DUMMY", Collections.emptyMap(), segments); final ImmutableDruidServer server = new ImmutableDruidServer( DUMMY_SERVER, @@ -118,6 +133,7 @@ public void registerSegmentCallback(Executor exec, final SegmentCallback callbac exec.execute(() -> callback.segmentAdded(DUMMY_SERVER_REALTIME, segment)); } exec.execute(callback::segmentViewInitialized); + segmentCallbackExecs.add(new Pair<>(exec, callback)); } @Override @@ -130,6 +146,7 @@ public void registerTimelineCallback(final Executor exec, final TimelineCallback exec.execute(() -> callback.segmentAdded(DUMMY_SERVER_REALTIME, segment)); } exec.execute(callback::timelineInitialized); + timelineCallbackExecs.add(new Pair<>(exec, callback)); } @Override @@ -143,4 +160,57 @@ public void registerServerRemovedCallback(Executor exec, ServerRemovedCallback c { // Do nothing } + + public void addSegment(DataSegment segment, ServerType serverType) + { + final Pair> whichServerAndSegments = + getDummyServerAndSegmentsForType(serverType); + final DruidServerMetadata whichServer = whichServerAndSegments.lhs; + whichServerAndSegments.rhs.add(segment); + segmentCallbackExecs.forEach( + execAndCallback -> execAndCallback.lhs.execute(() -> execAndCallback.rhs.segmentAdded(whichServer, segment)) + ); + timelineCallbackExecs.forEach( + execAndCallback -> execAndCallback.lhs.execute(() -> execAndCallback.rhs.segmentAdded(whichServer, segment)) + ); + } + + public void removeSegment(DataSegment segment, ServerType serverType) + { + final Pair> whichServerAndSegments = + getDummyServerAndSegmentsForType(serverType); + final DruidServerMetadata whichServer = whichServerAndSegments.lhs; + whichServerAndSegments.rhs.remove(segment); + segmentCallbackExecs.forEach( + execAndCallback -> execAndCallback.lhs.execute(() -> execAndCallback.rhs.segmentRemoved(whichServer, segment)) + ); + timelineCallbackExecs.forEach( + execAndCallback -> execAndCallback.lhs.execute(() -> { + execAndCallback.rhs.serverSegmentRemoved(whichServer, segment); + // assume that all replicas have been removed and fire this one too + execAndCallback.rhs.segmentRemoved(segment); + }) + ); + } + + private Pair> getDummyServerAndSegmentsForType(ServerType serverType) + { + final DruidServerMetadata whichServer; + final List whichSegments; + switch (serverType) { + case BROKER: + whichServer = DUMMY_BROKER; + whichSegments = brokerSegments; + break; + case REALTIME: + whichServer = DUMMY_SERVER_REALTIME; + whichSegments = realtimeSegments; + break; + default: + whichServer = DUMMY_SERVER; + whichSegments = segments; + break; + } + return new Pair<>(whichServer, whichSegments); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index 1f2f286d1a88..c4e2de59c95c 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -41,6 +41,7 @@ import org.apache.druid.query.QueryException; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryRunnerFactoryConglomerate; +import org.apache.druid.query.QueryUnsupportedException; import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.server.QueryCapacityExceededException; import org.apache.druid.server.QueryScheduler; @@ -717,6 +718,22 @@ public void testResourceLimitExceeded() throws Exception checkSqlRequestLog(false); } + @Test + public void testUnsupportedQueryThrowsException() throws Exception + { + String errorMessage = "This will be support in Druid 9999"; + SqlQuery badQuery = EasyMock.createMock(SqlQuery.class); + EasyMock.expect(badQuery.getQuery()).andReturn("SELECT ANSWER TO LIFE"); + EasyMock.expect(badQuery.getContext()).andReturn(ImmutableMap.of()); + EasyMock.expect(badQuery.getParameterList()).andThrow(new QueryUnsupportedException(errorMessage)); + EasyMock.replay(badQuery); + final QueryException exception = doPost(badQuery).lhs; + + Assert.assertNotNull(exception); + Assert.assertEquals(exception.getErrorCode(), QueryUnsupportedException.ERROR_CODE); + Assert.assertEquals(exception.getErrorClass(), QueryUnsupportedException.class.getName()); + } + @Test public void testTooManyRequests() throws Exception { diff --git a/sql/src/test/resources/calcite/tests/README.md b/sql/src/test/resources/calcite/tests/README.md new file mode 100644 index 000000000000..461679c1e53a --- /dev/null +++ b/sql/src/test/resources/calcite/tests/README.md @@ -0,0 +1,7 @@ +This package contains ingestion specs for datasources used by Calcite Tests. + +The purpose of these files is to make it easier to look at and manipulate the data under test so that you can easily +validate if the results of the SQL query written are as expected. + +> NOTE: The provided specs are not guaranteed to be in sync with the datasources used by the test. The source of truth +> is org.apache.druid.sql.calcite.util.CalciteTests diff --git a/sql/src/test/resources/calcite/tests/foo.json b/sql/src/test/resources/calcite/tests/foo.json new file mode 100644 index 000000000000..4b0d65b5d215 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/foo.json @@ -0,0 +1,53 @@ +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"t\":\"2000-01-01\",\"m1\":\"1.0\",\"m2\":\"1.0\",\"dim1\":\"\",\"dim2\":[\"a\"],\"dim3\":[\"a\",\"b\"]},\n{\"t\":\"2000-01-02\",\"m1\":\"2.0\",\"m2\":\"2.0\",\"dim1\":\"10.1\",\"dim2\":[],\"dim3\":[\"b\",\"c\"]},\n{\"t\":\"2000-01-03\",\"m1\":\"3.0\",\"m2\":\"3.0\",\"dim1\":\"2\",\"dim2\":[\"\"],\"dim3\":[\"d\"]},\n{\"t\":\"2001-01-01\",\"m1\":\"4.0\",\"m2\":\"4.0\",\"dim1\":\"1\",\"dim2\":[\"a\"],\"dim3\":[\"\"]},\n{\"t\":\"2001-01-02\",\"m1\":\"5.0\",\"m2\":\"5.0\",\"dim1\":\"def\",\"dim2\":[\"abc\"],\"dim3\":[]},\n{\"t\":\"2001-01-03\",\"m1\":\"6.0\",\"m2\":\"6.0\",\"dim1\":\"abc\"}" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "foo", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "NONE", + "rollup": false, + "segmentGranularity": "YEAR" + }, + "timestampSpec": { + "column": "t", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "dim1", + "dim2", + "dim3" + ] + }, + "metricsSpec": [ + { + "name": "m1", + "type": "floatSum", + "fieldName": "m1" + }, + { + "name": "m2", + "type": "doubleSum", + "fieldName": "m2" + } + ] + } + } +} \ No newline at end of file diff --git a/sql/src/test/resources/calcite/tests/foo2.json b/sql/src/test/resources/calcite/tests/foo2.json new file mode 100644 index 000000000000..d865c2ce549f --- /dev/null +++ b/sql/src/test/resources/calcite/tests/foo2.json @@ -0,0 +1,47 @@ +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"t\":\"2000-01-01\",\"dim1\":\"דרואיד\",\"dim2\":\"he\",\"m1\":1.0}\n{\"t\":\"2000-01-01\",\"dim1\":\"druid\",\"dim2\":\"en\",\"m1\":1.0}\n{\"t\":\"2000-01-01\",\"dim1\":\"друид\",\"dim2\":\"ru\",\"m1\":1.0}" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "foo2", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "NONE", + "rollup": false, + "segmentGranularity": "YEAR" + }, + "timestampSpec": { + "column": "t", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + "dim1", + "dim2" + ] + }, + "metricsSpec": [ + { + "name": "m1", + "type": "longSum", + "fieldName": "m1" + } + ] + } + } +} \ No newline at end of file diff --git a/sql/src/test/resources/calcite/tests/foo4.json b/sql/src/test/resources/calcite/tests/foo4.json new file mode 100644 index 000000000000..ddf275fc3ba6 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/foo4.json @@ -0,0 +1,52 @@ +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"t\":\"2000-01-01T10:51:45.695Z\",\"m1\":\"1.0\",\"m2\":\"1.0\",\"dim1\":\"\",\"dim2\":[\"a\"],\"dim3\":[\"a\",\"b\"]}\n{\"t\":\"2000-01-18T10:51:45.695Z\",\"m1\":\"2.0\",\"m2\":\"2.0\",\"dim1\":\"10.1\",\"dim2\":[],\"dim3\":[\"b\",\"c\"]}" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "foo4", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "HOUR", + "rollup": true, + "segmentGranularity": "YEAR" + }, + "timestampSpec": { + "column": "t", + "format": "iso" + }, + "dimensionsSpec": { + "dimensions": [ + "dim2", + "dim3" + ] + }, + "metricsSpec": [ + { + "name": "m1", + "type": "floatSum", + "fieldName": "m1" + }, + { + "name": "m2", + "type": "doubleSum", + "fieldName": "m2" + } + ] + } + } +} \ No newline at end of file diff --git a/sql/src/test/resources/calcite/tests/lookyloo.json b/sql/src/test/resources/calcite/tests/lookyloo.json new file mode 100644 index 000000000000..8514c3d38670 --- /dev/null +++ b/sql/src/test/resources/calcite/tests/lookyloo.json @@ -0,0 +1,6 @@ +{ + "a": "xa", + "abc": "xabc", + "nosuchkey": "mysteryvalue", + "6": "x6" +} \ No newline at end of file diff --git a/sql/src/test/resources/calcite/tests/numFoo.json b/sql/src/test/resources/calcite/tests/numFoo.json new file mode 100644 index 000000000000..f298b1db7fef --- /dev/null +++ b/sql/src/test/resources/calcite/tests/numFoo.json @@ -0,0 +1,79 @@ +{ + "type": "index_parallel", + "spec": { + "ioConfig": { + "type": "index_parallel", + "inputSource": { + "type": "inline", + "data": "{\"t\":\"2000-01-01\",\"m1\":\"1.0\",\"m2\":\"1.0\",\"d1\":1.0,\"f1\":1.0,\"l1\":7,\"dim1\":\"\",\"dim2\":[\"a\"],\"dim3\":[\"a\",\"b\"],\"dim4\":\"a\",\"dim5\":\"aa\"}\n{\"t\":\"2000-01-02\",\"m1\":\"2.0\",\"m2\":\"2.0\",\"d1\":1.7,\"d2\":1.7,\"f1\":0.1,\"f2\":0.1,\"l1\":325323,\"l2\":325323,\"dim1\":\"10.1\",\"dim2\":[],\"dim3\":[\"b\",\"c\"],\"dim4\":\"a\",\"dim5\":\"ab\"}\n{\"t\":\"2000-01-03\",\"m1\":\"3.0\",\"m2\":\"3.0\",\"d1\":0.0,\"d2\":0.0,\"f1\":0.0,\"f2\":0.0,\"l1\":0,\"l2\":0,\"dim1\":\"2\",\"dim2\":[\"\"],\"dim3\":[\"d\"],\"dim4\":\"a\",\"dim5\":\"ba\"}\n{\"t\":\"2001-01-01\",\"m1\":\"4.0\",\"m2\":\"4.0\",\"dim1\":\"1\",\"dim2\":[\"a\"],\"dim3\":[\"\"],\"dim4\":\"b\",\"dim5\":\"ad\"}\n{\"t\":\"2001-01-02\",\"m1\":\"5.0\",\"m2\":\"5.0\",\"dim1\":\"def\",\"dim2\":[\"abc\"],\"dim3\":[],\"dim4\":\"b\",\"dim5\":\"aa\"}\n{\"t\":\"2001-01-03\",\"m1\":\"6.0\",\"m2\":\"6.0\",\"dim1\":\"abc\",\"dim4\":\"b\",\"dim5\":\"ab\"}" + }, + "inputFormat": { + "type": "json" + } + }, + "tuningConfig": { + "type": "index_parallel", + "partitionsSpec": { + "type": "dynamic" + } + }, + "dataSchema": { + "dataSource": "numFoo", + "granularitySpec": { + "type": "uniform", + "queryGranularity": "NONE", + "rollup": false, + "segmentGranularity": "YEAR" + }, + "timestampSpec": { + "column": "t", + "format": "auto" + }, + "dimensionsSpec": { + "dimensions": [ + { + "type": "double", + "name": "d1" + }, + { + "type": "double", + "name": "d2" + }, + "dim1", + "dim2", + "dim3", + "dim4", + "dim5", + { + "type": "float", + "name": "f1" + }, + { + "type": "float", + "name": "f2" + }, + { + "type": "long", + "name": "l1" + }, + { + "type": "long", + "name": "l2" + } + ] + }, + "metricsSpec": [ + { + "name": "m1", + "type": "floatSum", + "fieldName": "m1" + }, + { + "name": "m2", + "type": "doubleSum", + "fieldName": "m2" + } + ] + } + } +} \ No newline at end of file diff --git a/web-console/script/druid b/web-console/script/druid index 96f67b856a89..767c3a70b9fe 100755 --- a/web-console/script/druid +++ b/web-console/script/druid @@ -58,13 +58,15 @@ function _download_zookeeper() { local dest="$1" local zk_version zk_version="$(_get_zookeeper_version)" + #zk_tar=apache-zookeeper-${zk_version}-bin # for zk 3.5.x + zk_tar=zookeeper-${zk_version} # for zk 3.4.x _log "Downloading zookeeper" - curl -s "https://archive.apache.org/dist/zookeeper/zookeeper-${zk_version}/zookeeper-${zk_version}.tar.gz" \ + curl -s "https://archive.apache.org/dist/zookeeper/zookeeper-${zk_version}/$zk_tar.tar.gz" \ | tar xz \ && rm -rf "$dest" \ - && mv "zookeeper-${zk_version}" "$dest" \ - && rm -f "zookeeper-${zk_version}" + && mv "$zk_tar" "$dest" \ + && rm -f "$zk_tar" } function _build_distribution() { diff --git a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap index 43d59de16916..869d8d020238 100644 --- a/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap +++ b/web-console/src/components/rule-editor/__snapshots__/rule-editor.spec.tsx.snap @@ -214,6 +214,186 @@ exports[`rule editor matches snapshot no tier in rule 1`] = ` `; +exports[`rule editor matches snapshot with broadcast rule 1`] = ` +
+
+ +
+ +
+
+
+
+
+
+
+
+ + + + + double-caret-vertical + + + + +
+
+ +
+
+
+
+
+
+
+
+`; + exports[`rule editor matches snapshot with existing tier and non existing tier in rule 1`] = `
{ const { container } = render(ruleEditor); expect(container.firstChild).toMatchSnapshot(); }); + + it('matches snapshot with broadcast rule', () => { + const ruleEditor = ( + {}} + onDelete={() => {}} + moveUp={null} + moveDown={null} + /> + ); + const { container } = render(ruleEditor); + expect(container.firstChild).toMatchSnapshot(); + }); }); diff --git a/web-console/src/components/rule-editor/rule-editor.tsx b/web-console/src/components/rule-editor/rule-editor.tsx index cf861c099598..09fca2fe8c5b 100644 --- a/web-console/src/components/rule-editor/rule-editor.tsx +++ b/web-console/src/components/rule-editor/rule-editor.tsx @@ -26,7 +26,6 @@ import { InputGroup, NumericInput, Switch, - TagInput, } from '@blueprintjs/core'; import { IconNames } from '@blueprintjs/icons'; import React, { useState } from 'react'; @@ -209,15 +208,6 @@ export const RuleEditor = React.memo(function RuleEditor(props: RuleEditorProps) {renderTierAdder()} )} - {RuleUtil.hasColocatedDataSources(rule) && ( - - onChange(RuleUtil.changeColocatedDataSources(rule, v))} - fill - /> - - )}
diff --git a/web-console/src/utils/load-rule.ts b/web-console/src/utils/load-rule.ts index ecd90306397e..6cae0a306cb4 100644 --- a/web-console/src/utils/load-rule.ts +++ b/web-console/src/utils/load-rule.ts @@ -36,7 +36,6 @@ export interface Rule { period?: string; includeFuture?: boolean; tieredReplicants?: Record; - colocatedDataSources?: string[]; } export class RuleUtil { @@ -83,8 +82,6 @@ export class RuleUtil { delete newRule.tieredReplicants; } - if (!RuleUtil.hasColocatedDataSources(newRule)) delete newRule.colocatedDataSources; - return newRule; } @@ -124,12 +121,4 @@ export class RuleUtil { const newTieredReplicants = deepSet(rule.tieredReplicants || {}, tier, replication); return deepSet(rule, 'tieredReplicants', newTieredReplicants); } - - static hasColocatedDataSources(rule: Rule): boolean { - return rule.type.startsWith('broadcast'); - } - - static changeColocatedDataSources(rule: Rule, colocatedDataSources: string[]): Rule { - return deepSet(rule, 'colocatedDataSources', colocatedDataSources); - } } diff --git a/website/.spelling b/website/.spelling index ce11862db740..f96b49dc6cfc 100644 --- a/website/.spelling +++ b/website/.spelling @@ -98,6 +98,7 @@ IndexTask InfluxDB InputFormat InputSource +InputSources Integer.MAX_VALUE JBOD JDBC @@ -151,6 +152,7 @@ S3 SDK SIGAR SPNEGO +SqlInputSource SQLServer SSD SSDs @@ -549,6 +551,10 @@ com.microsoft.sqlserver.jdbc.SQLServerDriver sqljdbc - ../docs/development/extensions-contrib/statsd.md convertRange +- ../docs/development/extensions-contrib/prometheus.md +HTTPServer +conversionFactor +prometheus - ../docs/development/extensions-contrib/tdigestsketch-quantiles.md postAggregator quantileFromTDigestSketch @@ -1069,6 +1075,7 @@ nextafter nvl parse_long regexp_extract +regexp_like result1 result2 rint @@ -1751,4 +1758,4 @@ UserGroupInformation CVE-2019-17571 CVE-2019-12399 CVE-2018-17196 -bin.tar.gz \ No newline at end of file +bin.tar.gz diff --git a/website/i18n/en.json b/website/i18n/en.json index 9561136a5e59..6c40a6157082 100644 --- a/website/i18n/en.json +++ b/website/i18n/en.json @@ -334,7 +334,7 @@ "title": "kubernetes" }, "operations/management-uis": { - "title": "Management UIs" + "title": "Legacy Management UIs" }, "operations/metadata-migration": { "title": "Metadata Migration" diff --git a/website/sidebars.json b/website/sidebars.json index da8a73458417..019f8eeef89e 100644 --- a/website/sidebars.json +++ b/website/sidebars.json @@ -111,7 +111,8 @@ "configuration/logging" ], "Operations": [ - "operations/management-uis", + "operations/druid-console", + "operations/getting-started", "operations/basic-cluster-tuning", "operations/api-reference", "operations/high-availability", @@ -131,10 +132,9 @@ "type": "subcategory", "label": "Misc", "ids": [ + "operations/management-uis", "operations/deep-storage-migration", - "operations/druid-console", "operations/export-metadata", - "operations/getting-started", "operations/metadata-migration", "operations/segment-optimization", "operations/use_sbt_to_build_fat_jar"