diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java index 73cba8c5d977..eef24302f5d6 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java @@ -26,12 +26,12 @@ import org.apache.druid.collections.bitmap.RoaringBitmapFactory; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.segment.column.BitmapColumnIndex; -import org.apache.druid.segment.column.IndexedUtf8ValueSetIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.IndexedUtf8ValueIndexes; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.serde.StringUtf8ColumnIndexSupplier; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; @@ -72,7 +72,7 @@ public class DictionaryEncodedStringIndexSupplierBenchmark public static class BenchmarkState { @Nullable - private IndexedUtf8ValueSetIndex stringValueSetIndex; + private IndexedUtf8ValueIndexes stringValueSetIndex; private final TreeSet values = new TreeSet<>(); private static final int START_INT = 10_000_000; @@ -112,7 +112,7 @@ public void setup() ); StringUtf8ColumnIndexSupplier indexSupplier = new StringUtf8ColumnIndexSupplier<>(bitmapFactory, dictionaryUtf8::singleThreaded, bitmaps, null); - stringValueSetIndex = (IndexedUtf8ValueSetIndex) indexSupplier.as(StringValueSetIndex.class); + stringValueSetIndex = (IndexedUtf8ValueIndexes) indexSupplier.as(StringValueSetIndexes.class); List filterValues = new ArrayList<>(); List nonFilterValues = new ArrayList<>(); for (int i = 0; i < dictionarySize; i++) { diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java index b304dba196a3..f42cade76737 100644 --- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java +++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java @@ -38,7 +38,6 @@ import org.apache.druid.query.aggregation.tdigestsketch.TDigestSketchToQuantilePostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; @@ -444,7 +443,7 @@ public void testEmptyTimeseriesResults() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .granularity(Granularities.ALL) .aggregators(ImmutableList.of( new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION), @@ -476,7 +475,7 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) @@ -484,11 +483,11 @@ public void testGroupByAggregatorDefaultValues() aggregators( new FilteredAggregatorFactory( new TDigestSketchAggregatorFactory("a0:agg", "m1", TDigestSketchAggregatorFactory.DEFAULT_COMPRESSION), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new TDigestSketchAggregatorFactory("a1:agg", "qsketch_m1", 100), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java index 35370648f0bf..6d42c678bbb1 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/HllSketchBuildAggregatorFactory.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.datasketches.hll.HllSketch; import org.apache.datasketches.hll.TgtHllType; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.Aggregator; @@ -221,6 +222,8 @@ private HllSketchUpdater formulateSketchUpdater(ColumnSelectorFactory columnSele } }; break; + case ARRAY: + throw InvalidInput.exception("ARRAY types are not supported for hll sketch"); default: updater = sketch -> { Object obj = selector.getObject(); diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java index aac55a2e0b72..bce83d50d5cf 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/hll/vector/HllSketchBuildVectorProcessorFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.query.aggregation.datasketches.hll.vector; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.StringEncoding; import org.apache.druid.query.aggregation.datasketches.hll.HllSketchBuildBufferAggregatorHelper; import org.apache.druid.segment.VectorColumnProcessorFactory; @@ -83,6 +84,15 @@ public HllSketchBuildVectorProcessor makeLongProcessor(ColumnCapabilities capabi return new LongHllSketchBuildVectorProcessor(helper, selector); } + @Override + public HllSketchBuildVectorProcessor makeArrayProcessor( + ColumnCapabilities capabilities, + VectorObjectSelector selector + ) + { + throw DruidException.defensive("ARRAY types are not supported for hll sketch"); + } + @Override public HllSketchBuildVectorProcessor makeObjectProcessor( ColumnCapabilities capabilities, diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java index 23207a596779..b18153067a2d 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllDoublesSketchAggregatorFactory.java @@ -179,7 +179,7 @@ public VectorAggregator makeSingleValueDimensionProcessor( SingleValueDimensionVectorSelector selector ) { - return new KllSketchNoOpBufferAggregator(getEmptySketch()); + return new KllSketchNoOpBufferAggregator<>(getEmptySketch()); } @Override @@ -188,7 +188,7 @@ public VectorAggregator makeMultiValueDimensionProcessor( MultiValueDimensionVectorSelector selector ) { - return new KllSketchNoOpBufferAggregator(getEmptySketch()); + return new KllSketchNoOpBufferAggregator<>(getEmptySketch()); } @Override @@ -209,6 +209,12 @@ public VectorAggregator makeLongProcessor(ColumnCapabilities capabilities, Vecto return new KllDoublesSketchBuildVectorAggregator(selector, getK(), getMaxIntermediateSizeWithNulls()); } + @Override + public VectorAggregator makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + return new KllSketchNoOpBufferAggregator<>(getEmptySketch()); + } + @Override public VectorAggregator makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java index 6b7f563c6750..c5506a3b86ab 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/kll/KllFloatsSketchAggregatorFactory.java @@ -179,7 +179,7 @@ public VectorAggregator makeSingleValueDimensionProcessor( SingleValueDimensionVectorSelector selector ) { - return new KllSketchNoOpBufferAggregator(getEmptySketch()); + return new KllSketchNoOpBufferAggregator<>(getEmptySketch()); } @Override @@ -188,7 +188,7 @@ public VectorAggregator makeMultiValueDimensionProcessor( MultiValueDimensionVectorSelector selector ) { - return new KllSketchNoOpBufferAggregator(getEmptySketch()); + return new KllSketchNoOpBufferAggregator<>(getEmptySketch()); } @Override @@ -209,6 +209,12 @@ public VectorAggregator makeLongProcessor(ColumnCapabilities capabilities, Vecto return new KllFloatsSketchBuildVectorAggregator(selector, getK(), getMaxIntermediateSizeWithNulls()); } + @Override + public VectorAggregator makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + return new KllSketchNoOpBufferAggregator<>(getEmptySketch()); + } + @Override public VectorAggregator makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java index c5442dfaf44f..5168a558440e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/quantiles/DoublesSketchAggregatorFactory.java @@ -208,6 +208,12 @@ public VectorAggregator makeLongProcessor(ColumnCapabilities capabilities, Vecto return new DoublesSketchBuildVectorAggregator(selector, k, getMaxIntermediateSizeWithNulls()); } + @Override + public VectorAggregator makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + return new NoopDoublesSketchBufferAggregator(); + } + @Override public VectorAggregator makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) { diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java index a52ff1819e76..211373e873b0 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactory.java @@ -27,6 +27,7 @@ import org.apache.datasketches.theta.SetOperation; import org.apache.datasketches.theta.Union; import org.apache.datasketches.thetacommon.ThetaUtil; +import org.apache.druid.error.InvalidInput; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.aggregation.AggregateCombiner; import org.apache.druid.query.aggregation.Aggregator; @@ -39,6 +40,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -78,6 +80,10 @@ public SketchAggregatorFactory(String name, String fieldName, Integer size, byte @Override public Aggregator factorize(ColumnSelectorFactory metricFactory) { + ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName); + if (capabilities != null && capabilities.isArray()) { + throw InvalidInput.exception("ARRAY types are not supported for theta sketch"); + } BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); return new SketchAggregator(selector, size); } @@ -85,6 +91,10 @@ public Aggregator factorize(ColumnSelectorFactory metricFactory) @Override public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) { + ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName); + if (capabilities != null && capabilities.isArray()) { + throw InvalidInput.exception("ARRAY types are not supported for theta sketch"); + } BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); final SketchAggregator aggregator = new SketchAggregator(selector, size); return new AggregatorAndSize(aggregator, aggregator.getInitialSizeBytes()); @@ -94,6 +104,10 @@ public AggregatorAndSize factorizeWithSize(ColumnSelectorFactory metricFactory) @Override public BufferAggregator factorizeBuffered(ColumnSelectorFactory metricFactory) { + ColumnCapabilities capabilities = metricFactory.getColumnCapabilities(fieldName); + if (capabilities != null && capabilities.isArray()) { + throw InvalidInput.exception("ARRAY types are not supported for theta sketch"); + } BaseObjectColumnValueSelector selector = metricFactory.makeColumnValueSelector(fieldName); return new SketchBufferAggregator(selector, size, getMaxIntermediateSizeWithNulls()); } diff --git a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactory.java b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactory.java index 2915f34a7930..37df64f0df2e 100644 --- a/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactory.java +++ b/extensions-core/datasketches/src/main/java/org/apache/druid/query/aggregation/datasketches/util/ToObjectVectorColumnProcessorFactory.java @@ -133,6 +133,12 @@ public Supplier makeLongProcessor(ColumnCapabilities capabilities, Vec }; } + @Override + public Supplier makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + return selector::getObjectVector; + } + @Override public Supplier makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) { 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 1431aa49a9a1..349f1a57d1c0 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 @@ -187,7 +187,7 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest private static final List EXPECTED_FILTERED_AGGREGATORS = EXPECTED_PA_AGGREGATORS.stream() .limit(5) - .map(factory -> new FilteredAggregatorFactory(factory, selector("dim2", "a", null))) + .map(factory -> new FilteredAggregatorFactory(factory, equality("dim2", "a", ColumnType.STRING))) .collect(Collectors.toList()); /** @@ -344,7 +344,7 @@ public void testApproxCountDistinctHllSketch() new HllSketchBuildAggregatorFactory("a1", "dim2", null, null, null, null, ROUND), new FilteredAggregatorFactory( new HllSketchBuildAggregatorFactory("a2", "dim2", null, null, null, null, ROUND), - BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null)) + not(equality("dim2", "", ColumnType.STRING)) ), new HllSketchBuildAggregatorFactory("a3", "v0", null, null, null, null, ROUND), new HllSketchBuildAggregatorFactory("a4", "v1", null, null, null, null, ROUND), @@ -436,7 +436,7 @@ public void testAvgDailyCountDistinctHllSketch() new LongSumAggregatorFactory("_a0:sum", "a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("_a0:count"), - BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null)) + notNull("a0") ) ) ) @@ -480,7 +480,7 @@ public void testApproxCountDistinctHllSketchIsRounded() new HllSketchBuildAggregatorFactory("a0", "m1", null, null, null, true, true) ) ) - .setHavingSpec(having(selector("a0", "2", null))) + .setHavingSpec(having(equality("a0", 2L, ColumnType.LONG))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -852,7 +852,7 @@ public void testEmptyTimeseriesResults() ImmutableList.of(Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .granularity(Granularities.ALL) .aggregators( aggregators( @@ -895,7 +895,7 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) @@ -911,7 +911,7 @@ public void testGroupByAggregatorDefaultValues() null, true ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new HllSketchBuildAggregatorFactory( @@ -923,7 +923,7 @@ public void testGroupByAggregatorDefaultValues() false, true ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) @@ -954,7 +954,7 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) @@ -962,11 +962,11 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches() aggregators( new FilteredAggregatorFactory( new HllSketchBuildAggregatorFactory("a0", "v0", null, null, null, null, true), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new HllSketchBuildAggregatorFactory("a1", "v0", null, null, null, null, true), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java index 184aba375a02..6800a2f61eb7 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -48,10 +48,7 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.NotDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; @@ -161,11 +158,11 @@ public void testQuantileOnFloatAndLongs() new DoublesSketchAggregatorFactory("a4:agg", "v0", null), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a5:agg", "m1", null), - new SelectorDimFilter("dim1", "abc", null) + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a6:agg", "m1", null), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + not(equality("dim1", "abc", ColumnType.STRING)) ), new DoublesSketchAggregatorFactory("a8:agg", "cnt", null) )) @@ -223,11 +220,11 @@ public void testQuantileOnComplexColumn() new DoublesSketchAggregatorFactory("a2:agg", "qsketch_m1", 256), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a4:agg", "qsketch_m1", null), - new SelectorDimFilter("dim1", "abc", null) + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a5:agg", "qsketch_m1", null), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + not(equality("dim1", "abc", ColumnType.STRING)) ) )) .postAggregators( @@ -325,11 +322,11 @@ public void testQuantileOnCastedString() new DoublesSketchAggregatorFactory("a4:agg", "v1", 128), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a5:agg", "v0", 128), - new SelectorDimFilter("dim2", "abc", null) + equality("dim2", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a6:agg", "v0", 128), - new NotDimFilter(new SelectorDimFilter("dim2", "abc", null)) + not(equality("dim2", "abc", ColumnType.STRING)) ) )) .postAggregators( @@ -728,7 +725,7 @@ public void testEmptyTimeseriesResults() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .aggregators(ImmutableList.of( new DoublesSketchAggregatorFactory("a0:agg", "m1", null), new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null), @@ -775,7 +772,7 @@ public void testEmptyTimeseriesResultsWithFinalizeSketches() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .aggregators(ImmutableList.of( new DoublesSketchAggregatorFactory("a0:agg", "m1", null), new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null), @@ -815,7 +812,7 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) @@ -823,19 +820,19 @@ public void testGroupByAggregatorDefaultValues() aggregators( new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a0:agg", "m1", null), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, false), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, false), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) @@ -886,7 +883,7 @@ public void testGroupByAggregatorDefaultValuesWithFinalizeSketches() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) @@ -894,19 +891,19 @@ public void testGroupByAggregatorDefaultValuesWithFinalizeSketches() aggregators( new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a0:agg", "m1", null), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a1:agg", "qsketch_m1", null), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a2:agg", "m1", null, null, true), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoublesSketchAggregatorFactory("a3:agg", "qsketch_m1", null, null, true), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java index d64b4263ac1b..1d70ff30f251 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/SketchAggregatorFactoryTest.java @@ -71,6 +71,7 @@ public void testFactorizeSized() ColumnSelectorFactory colSelectorFactory = EasyMock.mock(ColumnSelectorFactory.class); EasyMock.expect(colSelectorFactory.makeColumnValueSelector(EasyMock.anyString())) .andReturn(EasyMock.createMock(ColumnValueSelector.class)).anyTimes(); + EasyMock.expect(colSelectorFactory.getColumnCapabilities("x")).andReturn(null).anyTimes(); EasyMock.replay(colSelectorFactory); AggregatorAndSize aggregatorAndSize = AGGREGATOR_16384.factorizeWithSize(colSelectorFactory); 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 c1ddfa279d21..3946ce558b19 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 @@ -241,7 +241,7 @@ public void testApproxCountDistinctThetaSketch() null, null ), - BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("dim2", "", null)) + not(equality("dim2", "", ColumnType.STRING)) ), new SketchMergeAggregatorFactory( "a3", @@ -341,7 +341,7 @@ public void testAvgDailyCountDistinctThetaSketch() new LongSumAggregatorFactory("_a0:sum", "a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("_a0:count"), - BaseCalciteQueryTest.not(BaseCalciteQueryTest.selector("a0", null, null)) + notNull("a0") ) ) ) @@ -831,7 +831,7 @@ public void testEmptyTimeseriesResults() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .aggregators( ImmutableList.of( new SketchMergeAggregatorFactory( @@ -890,7 +890,7 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) @@ -905,7 +905,7 @@ public void testGroupByAggregatorDefaultValues() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new SketchMergeAggregatorFactory( @@ -916,7 +916,7 @@ public void testGroupByAggregatorDefaultValues() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new SketchMergeAggregatorFactory( @@ -927,7 +927,7 @@ public void testGroupByAggregatorDefaultValues() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new SketchMergeAggregatorFactory( @@ -938,7 +938,7 @@ public void testGroupByAggregatorDefaultValues() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) @@ -971,7 +971,7 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) @@ -986,7 +986,7 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new SketchMergeAggregatorFactory( @@ -997,7 +997,7 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new SketchMergeAggregatorFactory( @@ -1008,7 +1008,7 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new SketchMergeAggregatorFactory( @@ -1019,7 +1019,7 @@ public void testGroupByAggregatorDefaultValuesFinalizeOuterSketches() null, null ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java index fcf7099f28b0..1c77f0986e11 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java @@ -39,7 +39,6 @@ import org.apache.druid.query.extraction.SubstringDimExtractionFn; import org.apache.druid.query.filter.BloomKFilter; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; @@ -489,7 +488,7 @@ public void testEmptyTimeseriesResults() throws Exception .dataSource(CalciteTests.DATASOURCE3) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .filters(BaseCalciteQueryTest.bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .aggregators( ImmutableList.of( new BloomFilterAggregatorFactory( @@ -536,7 +535,7 @@ public void testGroupByAggregatorDefaultValues() throws Exception GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) @@ -548,7 +547,7 @@ public void testGroupByAggregatorDefaultValues() throws Exception new DefaultDimensionSpec("dim1", "a0:dim1"), TEST_NUM_ENTRIES ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new BloomFilterAggregatorFactory( @@ -556,7 +555,7 @@ public void testGroupByAggregatorDefaultValues() throws Exception new DefaultDimensionSpec("l1", "a1:l1", ColumnType.LONG), TEST_NUM_ENTRIES ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java index ca03eafb0a26..31d59a4423a8 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/expressions/BloomFilterExpressionsTest.java @@ -51,7 +51,7 @@ public class BloomFilterExpressionsTest extends InitializedNullHandlingTest ExprMacroTable macroTable = new ExprMacroTable(ImmutableList.of(createMacro, addMacro, testMacro)); Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers( - new ImmutableMap.Builder() + new ImmutableMap.Builder>() .put("bloomy", InputBindings.inputSupplier(BloomFilterExpressions.BLOOM_FILTER_TYPE, () -> new BloomKFilter(100))) .put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> SOME_STRING)) .put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> SOME_LONG)) diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java index 1b9089baf11d..26290da5578c 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java @@ -199,21 +199,24 @@ public void testSingleValueStringColumnWithNulls() throws IOException public void testMultiValueStringColumn() throws IOException { if (NullHandling.replaceWithDefault()) { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BloomDimFilter("dim2", bloomKFilter(1000, (String) null), null), ImmutableList.of("1", "2", "5") ); } else { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BloomDimFilter("dim2", bloomKFilter(1000, (String) null), null), ImmutableList.of("1", "5") ); - assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, ""), null), ImmutableList.of("2")); + assertFilterMatchesSkipArrays( + new BloomDimFilter("dim2", bloomKFilter(1000, ""), null), + ImmutableList.of("2") + ); } - assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "a"), null), ImmutableList.of("0", "3")); - assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "b"), null), ImmutableList.of("0")); - assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "c"), null), ImmutableList.of("4")); - assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "d"), null), ImmutableList.of()); + assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "a"), null), ImmutableList.of("0", "3")); + assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "b"), null), ImmutableList.of("0")); + assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "c"), null), ImmutableList.of("4")); + assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "d"), null), ImmutableList.of()); } @Test @@ -289,8 +292,8 @@ public void testSelectorWithLookupExtractionFn() throws IOException ImmutableList.of("0", "1", "2", "5") ); - assertFilterMatches(new BloomDimFilter("dim2", bloomKFilter(1000, "HELLO"), lookupFn), ImmutableList.of("0", "3")); - assertFilterMatches( + assertFilterMatchesSkipArrays(new BloomDimFilter("dim2", bloomKFilter(1000, "HELLO"), lookupFn), ImmutableList.of("0", "3")); + assertFilterMatchesSkipArrays( new BloomDimFilter("dim2", bloomKFilter(1000, "UNKNOWN"), lookupFn), ImmutableList.of("0", "1", "2", "4", "5") ); @@ -486,4 +489,17 @@ private static BloomKFilterHolder bloomKFilter(int expectedEntries, Long... valu } return BloomKFilterHolder.fromBloomKFilter(filter); } + + private static BloomKFilterHolder bloomKFilter(int expectedEntries, byte[]... values) throws IOException + { + BloomKFilter filter = new BloomKFilter(expectedEntries); + for (byte[] value : values) { + if (value == null) { + filter.addBytes(null, 0, 0); + } else { + filter.addBytes(value); + } + } + return BloomKFilterHolder.fromBloomKFilter(filter); + } } diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java index c5d6e631c410..15152694ce4d 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/sql/BloomDimFilterSqlTest.java @@ -84,6 +84,7 @@ public void testBloomFilter() throws IOException @Test public void testBloomFilterExprFilter() throws IOException { + cannotVectorize(); BloomKFilter filter = new BloomKFilter(1500); filter.addString("a-foo"); filter.addString("-foo"); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java index a4eae57756ce..e7b36163efa5 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java @@ -40,10 +40,7 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.NotDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; @@ -189,7 +186,7 @@ public void testQuantileOnFloatAndLongs() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - new SelectorDimFilter("dim1", "abc", null) + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new FixedBucketsHistogramAggregatorFactory( @@ -201,7 +198,7 @@ public void testQuantileOnFloatAndLongs() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + not(equality("dim1", "abc", ColumnType.STRING)) ), new FixedBucketsHistogramAggregatorFactory( "a8:agg", @@ -293,7 +290,7 @@ public void testQuantileOnCastedString() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - new SelectorDimFilter("dim1", "abc", null) + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new FixedBucketsHistogramAggregatorFactory( @@ -305,7 +302,7 @@ public void testQuantileOnCastedString() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + not(equality("dim1", "abc", ColumnType.STRING)) ) )) .postAggregators( @@ -408,7 +405,7 @@ public void testQuantileOnComplexColumn() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - new SelectorDimFilter("dim1", "abc", null) + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new FixedBucketsHistogramAggregatorFactory( @@ -420,7 +417,7 @@ public void testQuantileOnComplexColumn() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + not(equality("dim1", "abc", ColumnType.STRING)) ) )) .postAggregators( @@ -521,7 +518,7 @@ public void testEmptyTimeseriesResults() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .aggregators(ImmutableList.of( new FixedBucketsHistogramAggregatorFactory( "a0:agg", @@ -570,7 +567,7 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) @@ -586,7 +583,7 @@ public void testGroupByAggregatorDefaultValues() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new FixedBucketsHistogramAggregatorFactory( @@ -598,7 +595,7 @@ public void testGroupByAggregatorDefaultValues() FixedBucketsHistogram.OutlierHandlingMode.IGNORE, false ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index 4a67833b5249..53bc115c18b9 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -39,10 +39,7 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.NotDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; -import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndex; @@ -151,11 +148,11 @@ public void testQuantileOnFloatAndLongs() new ApproximateHistogramAggregatorFactory("a4:agg", "v0", null, null, null, null, false), new FilteredAggregatorFactory( new ApproximateHistogramAggregatorFactory("a5:agg", "m1", null, null, null, null, false), - new SelectorDimFilter("dim1", "abc", null) + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new ApproximateHistogramAggregatorFactory("a6:agg", "m1", null, null, null, null, false), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + not(equality("dim1", "abc", ColumnType.STRING)) ), new ApproximateHistogramAggregatorFactory("a8:agg", "cnt", null, null, null, null, false) )) @@ -208,15 +205,47 @@ public void testQuantileOnComplexColumn() .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) .aggregators(ImmutableList.of( - new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false), - new ApproximateHistogramFoldingAggregatorFactory("a2:agg", "hist_m1", 200, null, null, null, false), + new ApproximateHistogramFoldingAggregatorFactory( + "a0:agg", + "hist_m1", + null, + null, + null, + null, + false + ), + new ApproximateHistogramFoldingAggregatorFactory( + "a2:agg", + "hist_m1", + 200, + null, + null, + null, + false + ), new FilteredAggregatorFactory( - new ApproximateHistogramFoldingAggregatorFactory("a4:agg", "hist_m1", null, null, null, null, false), - new SelectorDimFilter("dim1", "abc", null) + new ApproximateHistogramFoldingAggregatorFactory( + "a4:agg", + "hist_m1", + null, + null, + null, + null, + false + ), + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( - new ApproximateHistogramFoldingAggregatorFactory("a5:agg", "hist_m1", null, null, null, null, false), - new NotDimFilter(new SelectorDimFilter("dim1", "abc", null)) + new ApproximateHistogramFoldingAggregatorFactory( + "a5:agg", + "hist_m1", + null, + null, + null, + null, + false + ), + not(equality("dim1", "abc", ColumnType.STRING)) ) )) .postAggregators( @@ -379,12 +408,21 @@ public void testEmptyTimeseriesResults() .dataSource(CalciteTests.DATASOURCE1) .intervals(new MultipleIntervalSegmentSpec(ImmutableList.of(Filtration.eternity()))) .granularity(Granularities.ALL) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) - .aggregators(ImmutableList.of( - new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false), - new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false) - - )) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) + .aggregators( + ImmutableList.of( + new ApproximateHistogramFoldingAggregatorFactory( + "a0:agg", + "hist_m1", + null, + null, + null, + null, + false + ), + new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false) + ) + ) .postAggregators( new QuantilePostAggregator("a0", "a0:agg", 0.01f), new QuantilePostAggregator("a1", "a1:agg", 0.01f) @@ -411,19 +449,35 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new FilteredAggregatorFactory( - new ApproximateHistogramFoldingAggregatorFactory("a0:agg", "hist_m1", null, null, null, null, false), - selector("dim1", "nonexistent", null) + new ApproximateHistogramFoldingAggregatorFactory( + "a0:agg", + "hist_m1", + null, + null, + null, + null, + false + ), + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( - new ApproximateHistogramAggregatorFactory("a1:agg", "m1", null, null, null, null, false), - selector("dim1", "nonexistent", null) + new ApproximateHistogramAggregatorFactory( + "a1:agg", + "m1", + null, + null, + null, + null, + false + ), + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index 0d4b3aff2f2d..6d444ac70de5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -60,8 +60,6 @@ import org.apache.druid.query.aggregation.post.FieldAccessPostAggregator; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.NotDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; import org.apache.druid.query.groupby.orderby.OrderByColumnSpec; @@ -421,7 +419,7 @@ public void testSelectOnFooWhereMatchesNoData() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Intervals.ETERNITY)) .columns("cnt", "dim1") - .filters(selector("dim2", "nonexistent", null)) + .filters(equality("dim2", "nonexistent", ColumnType.STRING)) .context(defaultScanQueryContext(context, resultSignature)) .build() ) @@ -455,7 +453,7 @@ public void testSelectAndOrderByOnFooWhereMatchesNoData() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Intervals.ETERNITY)) .columns("cnt", "dim1") - .filters(selector("dim2", "nonexistent", null)) + .filters(equality("dim2", "nonexistent", ColumnType.STRING)) .context(defaultScanQueryContext(context, resultSignature)) .orderBy(ImmutableList.of(new ScanQuery.OrderBy("dim1", ScanQuery.Order.ASCENDING))) .build() @@ -771,7 +769,7 @@ public void testJoinWithLookup() ) ) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(not(selector("j0.v", "xa", null))) + .setDimFilter(not(equality("j0.v", "xa", ColumnType.STRING))) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) @@ -958,7 +956,7 @@ private void testJoin(final JoinAlgorithm joinAlgorithm) new DoubleSumAggregatorFactory("a0:sum", "m2"), new FilteredAggregatorFactory( new CountAggregatorFactory("a0:count"), - not(selector("m2", null, null)), + notNull("m2"), // Not sure why the name is only set in SQL-compatible null mode. Seems strange. // May be due to JSON serialization: name is set on the serialized aggregator even @@ -1540,14 +1538,13 @@ public void testHavingOnApproximateCountDistinct() ) .setHavingSpec( having( - bound( + range( "a0", - "1", + ColumnType.LONG, + 1L, null, true, - false, - null, - StringComparators.NUMERIC + false ) ) ) @@ -1859,7 +1856,7 @@ public void testGroupByMultiValueMeasureQuery() aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - new NotDimFilter(new SelectorDimFilter("dim3", null, null)), + notNull("dim3"), "a0" ) ) @@ -2073,7 +2070,7 @@ public void testJoinUsesDifferentAlgorithm() .setAggregatorSpecs( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - new SelectorDimFilter("j0.d1", null, null), + isNull("j0.d1"), "a0" ) ) diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java index 5c496c466351..8f4f88e5d924 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java @@ -518,7 +518,7 @@ public void testEmptyTimeseriesResults() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .aggregators( new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"), new VarianceAggregatorFactory("a1:agg", "d1", "sample", "double"), @@ -568,7 +568,7 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) @@ -576,35 +576,35 @@ public void testGroupByAggregatorDefaultValues() aggregators( new FilteredAggregatorFactory( new VarianceAggregatorFactory("a0:agg", "d1", "population", "double"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new VarianceAggregatorFactory("a1:agg", "d1", "sample", "double"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new VarianceAggregatorFactory("a2:agg", "d1", "sample", "double"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new VarianceAggregatorFactory("a3:agg", "d1", "sample", "double"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new VarianceAggregatorFactory("a4:agg", "l1", "population", "long"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new VarianceAggregatorFactory("a5:agg", "l1", "sample", "long"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new VarianceAggregatorFactory("a6:agg", "l1", "sample", "long"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new VarianceAggregatorFactory("a7:agg", "l1", "sample", "long"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) ) ) diff --git a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java index c3b1a102f3de..d8dc51f6c245 100644 --- a/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java +++ b/extensions-core/testing-tools/src/test/java/org/apache/druid/query/sql/SleepSqlTest.java @@ -25,8 +25,6 @@ import org.apache.druid.guice.SleepModule; import org.apache.druid.query.Druids; import org.apache.druid.query.TableDataSource; -import org.apache.druid.query.filter.BoundDimFilter; -import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery.ResultFormat; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; @@ -61,7 +59,7 @@ public void testSleepFunction() ) ) .columns("v0") - .filters(new BoundDimFilter("m1", null, "2.0", null, true, null, null, StringComparators.NUMERIC)) + .filters(range("m1", ColumnType.DOUBLE, null, 2.0, false, true)) .resultFormat(ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) .context(QUERY_CONTEXT_DEFAULT) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java index decbf3b4038a..d048cf4d8b91 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidSegmentReader.java @@ -52,6 +52,7 @@ import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.VirtualColumns; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; @@ -60,6 +61,7 @@ import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.util.ArrayList; @@ -276,6 +278,15 @@ public Supplier makeLongProcessor(BaseLongColumnValueSelector selector) return () -> selector.isNull() ? null : selector.getLong(); } + @Override + public Supplier makeArrayProcessor( + BaseObjectColumnValueSelector selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + return selector::getObject; + } + @Override public Supplier makeComplexProcessor(BaseObjectColumnValueSelector selector) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java index 5580bfc5cc0c..09cec378b097 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/input/DruidSegmentReaderTest.java @@ -51,6 +51,7 @@ import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexSpec; @@ -683,6 +684,118 @@ public void close() Assert.assertTrue("Sequence is not closed", isSequenceClosed.booleanValue()); } + @Test + public void testArrayColumns() throws IOException + { + // make our own stuff here so that we don't pollute the shared spec, rows, and segment defined in setup and + // break all the other tests + DimensionsSpec dimensionsSpec = new DimensionsSpec( + ImmutableList.of( + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol"), + new AutoTypeColumnSchema("arrayCol") + ) + ); + List metrics = ImmutableList.of( + new CountAggregatorFactory("cnt"), + new HyperUniquesAggregatorFactory("met_s", "strCol") + ); + final List rows = ImmutableList.of( + new MapBasedInputRow( + DateTimes.of("2000"), + ImmutableList.of("strCol", "dblCol", "arrayCol"), + ImmutableMap.builder() + .put("strCol", "foo") + .put("dblCol", 1.23) + .put("arrayCol", ImmutableList.of("a", "b", "c")) + .build() + ), + new MapBasedInputRow( + DateTimes.of("2000T01"), + ImmutableList.of("strCol", "dblCol", "arrayCol"), + ImmutableMap.builder() + .put("strCol", "bar") + .put("dblCol", 4.56) + .put("arrayCol", ImmutableList.of("x", "y", "z")) + .build() + ) + ); + + InputStats inputStats = new InputStatsImpl(); + final IncrementalIndex incrementalIndex = + IndexBuilder.create() + .schema( + new IncrementalIndexSchema.Builder() + .withDimensionsSpec(dimensionsSpec) + .withMetrics(metrics.toArray(new AggregatorFactory[0])) + .withRollup(false) + .build() + ) + .rows(rows) + .buildIncrementalIndex(); + + File segmentDirectory = temporaryFolder.newFolder(); + long segmentSize; + try { + TestHelper.getTestIndexMergerV9( + OnHeapMemorySegmentWriteOutMediumFactory.instance() + ).persist( + incrementalIndex, + segmentDirectory, + IndexSpec.DEFAULT, + null + ); + segmentSize = FileUtils.getFileSize(segmentDirectory); + } + finally { + incrementalIndex.close(); + } + InputEntity entity = new BytesCountingInputEntity( + makeInputEntity( + Intervals.of("2000/P1D"), + segmentDirectory, + ImmutableList.of("strCol", "dblCol", "arrayCol"), + ImmutableList.of("cnt", "met_s") + ), + inputStats + ); + final DruidSegmentReader reader = new DruidSegmentReader( + entity, + indexIO, + new TimestampSpec("__time", "millis", DateTimes.of("1971")), + new DimensionsSpec( + ImmutableList.of( + StringDimensionSchema.create("strCol"), + new DoubleDimensionSchema("dblCol"), + new AutoTypeColumnSchema("arrayCol") + ) + ), + ColumnsFilter.all(), + null, + temporaryFolder.newFolder() + ); + + List readRows = readRows(reader); + + Assert.assertEquals(ImmutableList.of("strCol", "dblCol", "arrayCol"), readRows.get(0).getDimensions()); + Assert.assertEquals(DateTimes.of("2000T").getMillis(), readRows.get(0).getTimestampFromEpoch()); + Assert.assertEquals("foo", readRows.get(0).getRaw("strCol")); + Assert.assertEquals(1.23, readRows.get(0).getRaw("dblCol")); + Assert.assertArrayEquals(new Object[]{"a", "b", "c"}, (Object[]) readRows.get(0).getRaw("arrayCol")); + Assert.assertEquals(1L, readRows.get(0).getRaw("cnt")); + Assert.assertEquals(makeHLLC("foo"), readRows.get(0).getRaw("met_s")); + + Assert.assertEquals(DateTimes.of("2000T1").getMillis(), readRows.get(1).getTimestampFromEpoch()); + Assert.assertEquals("bar", readRows.get(1).getRaw("strCol")); + Assert.assertEquals(4.56, readRows.get(1).getRaw("dblCol")); + Assert.assertArrayEquals(new Object[]{"x", "y", "z"}, (Object[]) readRows.get(1).getRaw("arrayCol")); + Assert.assertEquals(1L, readRows.get(1).getRaw("cnt")); + Assert.assertEquals(makeHLLC("bar"), readRows.get(1).getRaw("met_s")); + + Assert.assertEquals(segmentSize, inputStats.getProcessedBytes()); + + } + private InputEntity makeInputEntity(final Interval interval) { return new BytesCountingInputEntity( diff --git a/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java b/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java index de1de837818e..a65999c1ae99 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/BinaryLogicalOperatorExpr.java @@ -50,6 +50,14 @@ protected boolean evalString(@Nullable String left, @Nullable String right) return Comparators.naturalNullsFirst().compare(left, right) < 0; } + @Override + protected boolean evalArray(ExprEval left, ExprEval right) + { + ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type()); + // type cannot be null here because ExprEval type is not nullable + return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) < 0; + } + @Override protected final boolean evalLong(long left, long right) { @@ -63,12 +71,6 @@ protected final boolean evalDouble(double left, double right) return Double.compare(left, right) < 0; } - @Override - public boolean canVectorize(InputBindingInspector inspector) - { - return inspector.canVectorize(left, right); - } - @Override public ExprVectorProcessor asVectorProcessor(VectorInputBindingInspector inspector) { @@ -96,6 +98,14 @@ protected boolean evalString(@Nullable String left, @Nullable String right) return Comparators.naturalNullsFirst().compare(left, right) <= 0; } + @Override + protected boolean evalArray(ExprEval left, ExprEval right) + { + ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type()); + // type cannot be null here because ExprEval type is not nullable + return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) <= 0; + } + @Override protected final boolean evalLong(long left, long right) { @@ -109,12 +119,6 @@ protected final boolean evalDouble(double left, double right) return Double.compare(left, right) <= 0; } - @Override - public boolean canVectorize(InputBindingInspector inspector) - { - return inspector.canVectorize(left, right); - } - @Override public ExprVectorProcessor asVectorProcessor(VectorInputBindingInspector inspector) { @@ -142,6 +146,14 @@ protected boolean evalString(@Nullable String left, @Nullable String right) return Comparators.naturalNullsFirst().compare(left, right) > 0; } + @Override + protected boolean evalArray(ExprEval left, ExprEval right) + { + ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type()); + // type cannot be null here because ExprEval type is not nullable + return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) > 0; + } + @Override protected final boolean evalLong(long left, long right) { @@ -155,12 +167,6 @@ protected final boolean evalDouble(double left, double right) return Double.compare(left, right) > 0; } - @Override - public boolean canVectorize(InputBindingInspector inspector) - { - return inspector.canVectorize(left, right); - } - @Override public ExprVectorProcessor asVectorProcessor(VectorInputBindingInspector inspector) { @@ -188,6 +194,14 @@ protected boolean evalString(@Nullable String left, @Nullable String right) return Comparators.naturalNullsFirst().compare(left, right) >= 0; } + @Override + protected boolean evalArray(ExprEval left, ExprEval right) + { + ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type()); + // type cannot be null here because ExprEval type is not nullable + return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) >= 0; + } + @Override protected final boolean evalLong(long left, long right) { @@ -201,12 +215,6 @@ protected final boolean evalDouble(double left, double right) return Double.compare(left, right) >= 0; } - @Override - public boolean canVectorize(InputBindingInspector inspector) - { - return inspector.canVectorize(left, right); - } - @Override public ExprVectorProcessor asVectorProcessor(VectorInputBindingInspector inspector) { @@ -235,21 +243,23 @@ protected boolean evalString(@Nullable String left, @Nullable String right) } @Override - protected final boolean evalLong(long left, long right) + protected boolean evalArray(ExprEval left, ExprEval right) { - return left == right; + ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type()); + // type cannot be null here because ExprEval type is not nullable + return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) == 0; } @Override - protected final boolean evalDouble(double left, double right) + protected final boolean evalLong(long left, long right) { return left == right; } @Override - public boolean canVectorize(InputBindingInspector inspector) + protected final boolean evalDouble(double left, double right) { - return inspector.canVectorize(left, right); + return left == right; } @Override @@ -280,21 +290,23 @@ protected boolean evalString(@Nullable String left, @Nullable String right) } @Override - protected final boolean evalLong(long left, long right) + protected boolean evalArray(ExprEval left, ExprEval right) { - return left != right; + ExpressionType type = ExpressionTypeConversion.leastRestrictiveType(left.type(), right.type()); + // type cannot be null here because ExprEval type is not nullable + return type.getNullableStrategy().compare(left.castTo(type).asArray(), right.castTo(type).asArray()) != 0; } @Override - protected final boolean evalDouble(double left, double right) + protected final boolean evalLong(long left, long right) { return left != right; } @Override - public boolean canVectorize(InputBindingInspector inspector) + protected final boolean evalDouble(double left, double right) { - return inspector.canVectorize(left, right); + return left != right; } @Override diff --git a/processing/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java b/processing/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java index 64acb04bb58f..8dd4b9602518 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java +++ b/processing/src/main/java/org/apache/druid/math/expr/BinaryOperatorExpr.java @@ -195,6 +195,9 @@ public ExprEval eval(ObjectBinding bindings) case LONG: result = evalLong(leftVal.asLong(), rightVal.asLong()); break; + case ARRAY: + result = evalArray(leftVal, rightVal); + break; case DOUBLE: default: if (NullHandling.sqlCompatible() && (leftVal.isNumericNull() || rightVal.isNumericNull())) { @@ -203,21 +206,20 @@ public ExprEval eval(ObjectBinding bindings) result = evalDouble(leftVal.asDouble(), rightVal.asDouble()); break; } - if (!ExpressionProcessing.useStrictBooleans() && !type.is(ExprType.STRING)) { + if (!ExpressionProcessing.useStrictBooleans() && !type.is(ExprType.STRING) && !type.isArray()) { return ExprEval.ofBoolean(result, type.getType()); } return ExprEval.ofLongBoolean(result); } - protected boolean evalString(@Nullable String left, @Nullable String right) - { - throw new IllegalArgumentException("unsupported type " + ExprType.STRING); - } + protected abstract boolean evalString(@Nullable String left, @Nullable String right); protected abstract boolean evalLong(long left, long right); protected abstract boolean evalDouble(double left, double right); + protected abstract boolean evalArray(ExprEval left, ExprEval right); + @Nullable @Override public ExpressionType getOutputType(InputBindingInspector inspector) @@ -228,4 +230,13 @@ public ExpressionType getOutputType(InputBindingInspector inspector) } return implicitCast; } + + @Override + public boolean canVectorize(InputBindingInspector inspector) + { + ExpressionType leftType = left.getOutputType(inspector); + ExpressionType rightType = right.getOutputType(inspector); + ExpressionType commonType = ExpressionTypeConversion.leastRestrictiveType(leftType, rightType); + return inspector.canVectorize(left, right) && (commonType == null || commonType.isPrimitive()); + } } diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java index eb0e0219463d..faa17d45e8d7 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionTypeConversion.java @@ -58,6 +58,12 @@ public static ExpressionType autoDetect(ExprEval eval, ExprEval otherEval) if (Types.is(type, ExprType.STRING) && Types.is(otherType, ExprType.STRING)) { return ExpressionType.STRING; } + // to preserve backwards compatibility, like with strings, we only use array type if both types are + // arrays... this is pretty wack, but it is what it is. we might want to consider changing this + // behavior in the future with a flag + if (type.isArray() && otherType.isArray()) { + return leastRestrictiveType(type, otherType); + } type = eval.value() != null ? type : otherType; otherType = otherEval.value() != null ? otherType : type; diff --git a/processing/src/main/java/org/apache/druid/math/expr/InputBindings.java b/processing/src/main/java/org/apache/druid/math/expr/InputBindings.java index d1bc2be32ce4..469ea05706be 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/InputBindings.java +++ b/processing/src/main/java/org/apache/druid/math/expr/InputBindings.java @@ -255,7 +255,7 @@ public static InputSupplier inputSupplier(ExpressionType type, Supplier bindings) + public static Expr.ObjectBinding forInputSuppliers(final Map> bindings) { return new Expr.ObjectBinding() { diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/vector/CardinalityVectorProcessorFactory.java b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/vector/CardinalityVectorProcessorFactory.java index c69a06520986..21b1d493fbab 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/vector/CardinalityVectorProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/cardinality/vector/CardinalityVectorProcessorFactory.java @@ -19,6 +19,7 @@ package org.apache.druid.query.aggregation.cardinality.vector; +import org.apache.druid.java.util.common.UOE; import org.apache.druid.segment.VectorColumnProcessorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; @@ -66,6 +67,15 @@ public CardinalityVectorProcessor makeLongProcessor(ColumnCapabilities capabilit return new LongCardinalityVectorProcessor(selector); } + @Override + public CardinalityVectorProcessor makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + throw new UOE( + "Cardinality aggregator does not support[%s] inputs", + capabilities.toColumnType() + ); + } + @Override public CardinalityVectorProcessor makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/BoundDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/BoundDimFilter.java index 1e317c5fbcc2..17cecc411511 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/BoundDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/BoundDimFilter.java @@ -279,8 +279,12 @@ && getExtractionFn() == null } else if (getUpper() == null) { range = isLowerStrict() ? Range.greaterThan(getLower()) : Range.atLeast(getLower()); } else { - range = Range.range(getLower(), isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED, - getUpper(), isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED); + range = Range.range( + getLower(), + isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED, + getUpper(), + isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED + ); } retSet.add(range); return retSet; @@ -323,7 +327,8 @@ public int hashCode() upperStrict, extractionFn, ordering, - filterTuning); + filterTuning + ); } @Override @@ -358,97 +363,212 @@ public String toString() return builder.appendFilterTuning(filterTuning).build(); } - private DruidLongPredicate createLongPredicate() + private Supplier makeLongPredicateSupplier() { - boolean hasLowerLongBound; - boolean hasUpperLongBound; - long lowerLongBound; - long upperLongBound; - boolean matchesNothing = false; - - if (hasLowerBound()) { - final Long lowerLong = GuavaUtils.tryParseLong(lower); - if (lowerLong == null) { - BigDecimal lowerBigDecimal = getBigDecimalLowerBoundFromFloatString(lower); - if (lowerBigDecimal == null) { - // Unparseable values fall before all actual numbers, so all numbers - // will match the lower bound. - hasLowerLongBound = false; - lowerLongBound = 0L; - } else { - try { - lowerLongBound = lowerBigDecimal.longValueExact(); - hasLowerLongBound = true; - } - catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long + return Suppliers.memoize(() -> { + boolean hasLowerLongBound; + boolean hasUpperLongBound; + long lowerLongBound; + long upperLongBound; + boolean matchesNothing = false; + + if (hasLowerBound()) { + final Long lowerLong = GuavaUtils.tryParseLong(lower); + if (lowerLong == null) { + BigDecimal lowerBigDecimal = getBigDecimalLowerBoundFromFloatString(lower); + if (lowerBigDecimal == null) { + // Unparseable values fall before all actual numbers, so all numbers + // will match the lower bound. hasLowerLongBound = false; lowerLongBound = 0L; - if (lowerBigDecimal.compareTo(BigDecimal.ZERO) > 0) { - // positive lower bound, > all longs, will match nothing - matchesNothing = true; + } else { + try { + lowerLongBound = lowerBigDecimal.longValueExact(); + hasLowerLongBound = true; + } + catch (ArithmeticException ae) { // the BigDecimal can't be contained in a long + hasLowerLongBound = false; + lowerLongBound = 0L; + if (lowerBigDecimal.compareTo(BigDecimal.ZERO) > 0) { + // positive lower bound, > all longs, will match nothing + matchesNothing = true; + } } } + } else { + hasLowerLongBound = true; + lowerLongBound = lowerLong; } } else { - hasLowerLongBound = true; - lowerLongBound = lowerLong; + hasLowerLongBound = false; + lowerLongBound = 0L; } - } else { - hasLowerLongBound = false; - lowerLongBound = 0L; - } - if (hasUpperBound()) { - Long upperLong = GuavaUtils.tryParseLong(upper); - if (upperLong == null) { - BigDecimal upperBigDecimal = getBigDecimalUpperBoundFromFloatString(upper); - if (upperBigDecimal == null) { - // Unparseable values fall before all actual numbers, so no numbers - // can match the upper bound. - matchesNothing = true; - hasUpperLongBound = false; - upperLongBound = 0L; - } else { - try { - upperLongBound = upperBigDecimal.longValueExact(); - hasUpperLongBound = true; - } - catch (ArithmeticException ae) { // the BigDecimal can't be - // contained in a long + if (hasUpperBound()) { + Long upperLong = GuavaUtils.tryParseLong(upper); + if (upperLong == null) { + BigDecimal upperBigDecimal = getBigDecimalUpperBoundFromFloatString(upper); + if (upperBigDecimal == null) { + // Unparseable values fall before all actual numbers, so no numbers + // can match the upper bound. + matchesNothing = true; hasUpperLongBound = false; upperLongBound = 0L; - if (upperBigDecimal.compareTo(BigDecimal.ZERO) < 0) { - // negative upper bound, < all longs, will match nothing - matchesNothing = true; + } else { + try { + upperLongBound = upperBigDecimal.longValueExact(); + hasUpperLongBound = true; + } + catch (ArithmeticException ae) { // the BigDecimal can't be + // contained in a long + hasUpperLongBound = false; + upperLongBound = 0L; + if (upperBigDecimal.compareTo(BigDecimal.ZERO) < 0) { + // negative upper bound, < all longs, will match nothing + matchesNothing = true; + } } } + } else { + hasUpperLongBound = true; + upperLongBound = upperLong; } } else { - hasUpperLongBound = true; - upperLongBound = upperLong; + hasUpperLongBound = false; + upperLongBound = 0L; } - } else { - hasUpperLongBound = false; - upperLongBound = 0L; - } - if (matchesNothing) { - return DruidLongPredicate.ALWAYS_FALSE; - } else { - return makeLongPredicateFromBounds( - hasLowerLongBound, - hasUpperLongBound, - lowerStrict, - upperStrict, - lowerLongBound, - upperLongBound); - } + if (matchesNothing) { + return DruidLongPredicate.ALWAYS_FALSE; + } else { + final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of( + hasLowerLongBound, + lowerStrict, + hasUpperLongBound, + upperStrict + ); + return RangeFilter.makeLongPredicate(rangeType, lowerLongBound, upperLongBound); + } + }); } - private Supplier makeLongPredicateSupplier() + private Supplier makeFloatPredicateSupplier() + { + return Suppliers.memoize(() -> { + final boolean hasLowerFloatBound; + final boolean hasUpperFloatBound; + final float lowerFloatBound; + final float upperFloatBound; + boolean matchesNothing = false; + + if (hasLowerBound()) { + final Float lowerFloat = Floats.tryParse(lower); + if (lowerFloat == null) { + // Unparseable values fall before all actual numbers, so all numbers + // will match the lower bound. + hasLowerFloatBound = false; + lowerFloatBound = 0L; + } else { + hasLowerFloatBound = true; + lowerFloatBound = lowerFloat; + } + } else { + hasLowerFloatBound = false; + lowerFloatBound = 0L; + } + + if (hasUpperBound()) { + Float upperFloat = Floats.tryParse(upper); + if (upperFloat == null) { + // Unparseable values fall before all actual numbers, so no numbers + // can match the upper bound. + matchesNothing = true; + hasUpperFloatBound = false; + upperFloatBound = 0L; + } else { + hasUpperFloatBound = true; + upperFloatBound = upperFloat; + } + } else { + hasUpperFloatBound = false; + upperFloatBound = 0L; + } + + + if (matchesNothing) { + return DruidFloatPredicate.ALWAYS_FALSE; + } else { + + final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of( + hasLowerFloatBound, + lowerStrict, + hasUpperFloatBound, + upperStrict + ); + final DruidDoublePredicate doublePredicate = RangeFilter.makeDoublePredicate( + rangeType, + lowerFloatBound, + upperFloatBound + ); + return doublePredicate::applyDouble; + } + }); + } + + private Supplier makeDoublePredicateSupplier() { - Supplier longPredicate = () -> createLongPredicate(); - return Suppliers.memoize(longPredicate); + return Suppliers.memoize(() -> { + final boolean hasLowerBound; + final boolean hasUpperBound; + final double lowerDoubleBound; + final double upperDoubleBound; + boolean matchesNothing = false; + + if (hasLowerBound()) { + final Double lowerDouble = Doubles.tryParse(lower); + if (lowerDouble == null) { + // Unparseable values fall before all actual numbers, so all numbers + // will match the lower bound. + hasLowerBound = false; + lowerDoubleBound = 0L; + } else { + hasLowerBound = true; + lowerDoubleBound = lowerDouble; + } + } else { + hasLowerBound = false; + lowerDoubleBound = 0L; + } + + if (hasUpperBound()) { + Double upperDouble = Doubles.tryParse(upper); + if (upperDouble == null) { + // Unparseable values fall before all actual numbers, so no numbers can + // match the upper bound. + matchesNothing = true; + hasUpperBound = false; + upperDoubleBound = 0L; + } else { + hasUpperBound = true; + upperDoubleBound = upperDouble; + } + } else { + hasUpperBound = false; + upperDoubleBound = 0L; + } + + if (matchesNothing) { + return DruidDoublePredicate.ALWAYS_FALSE; + } else { + final RangeFilter.RangeType rangeType = RangeFilter.RangeType.of( + hasLowerBound, + lowerStrict, + hasUpperBound, + upperStrict + ); + return RangeFilter.makeDoublePredicate(rangeType, lowerDoubleBound, upperDoubleBound); + } + }); } @Nullable @@ -486,225 +606,4 @@ private BigDecimal getBigDecimalUpperBoundFromFloatString(String floatStr) return convertedBD.setScale(0, RoundingMode.FLOOR); } } - - private DruidFloatPredicate createDruidFloatPredicate() - { - final boolean hasLowerFloatBound; - final boolean hasUpperFloatBound; - final float lowerFloatBound; - final float upperFloatBound; - boolean matchesNothing = false; - - if (hasLowerBound()) { - final Float lowerFloat = Floats.tryParse(lower); - if (lowerFloat == null) { - // Unparseable values fall before all actual numbers, so all numbers - // will match the lower bound. - hasLowerFloatBound = false; - lowerFloatBound = 0L; - } else { - hasLowerFloatBound = true; - lowerFloatBound = lowerFloat; - } - } else { - hasLowerFloatBound = false; - lowerFloatBound = 0L; - } - - if (hasUpperBound()) { - Float upperFloat = Floats.tryParse(upper); - if (upperFloat == null) { - // Unparseable values fall before all actual numbers, so no numbers - // can match the upper bound. - matchesNothing = true; - hasUpperFloatBound = false; - upperFloatBound = 0L; - } else { - hasUpperFloatBound = true; - upperFloatBound = upperFloat; - } - } else { - hasUpperFloatBound = false; - upperFloatBound = 0L; - } - - if (matchesNothing) { - return DruidFloatPredicate.ALWAYS_FALSE; - } else { - return input -> { - final DruidDoublePredicate druidDoublePredicate = makeDoublePredicateFromBounds( - hasLowerFloatBound, - hasUpperFloatBound, - lowerStrict, - upperStrict, - (double) lowerFloatBound, - (double) upperFloatBound); - return druidDoublePredicate.applyDouble((double) input); - }; - } - } - - private Supplier makeFloatPredicateSupplier() - { - Supplier floatPredicate = () -> createDruidFloatPredicate(); - return Suppliers.memoize(floatPredicate); - } - - private DruidDoublePredicate createDruidDoublePredicate() - { - final boolean hasLowerBound; - final boolean hasUpperBound; - final double lowerDoubleBound; - final double upperDoubleBound; - boolean matchesNothing = false; - - if (hasLowerBound()) { - final Double lowerDouble = Doubles.tryParse(lower); - if (lowerDouble == null) { - // Unparseable values fall before all actual numbers, so all numbers - // will match the lower bound. - hasLowerBound = false; - lowerDoubleBound = 0L; - } else { - hasLowerBound = true; - lowerDoubleBound = lowerDouble; - } - } else { - hasLowerBound = false; - lowerDoubleBound = 0L; - } - - if (hasUpperBound()) { - Double upperDouble = Doubles.tryParse(upper); - if (upperDouble == null) { - // Unparseable values fall before all actual numbers, so no numbers can - // match the upper bound. - matchesNothing = true; - hasUpperBound = false; - upperDoubleBound = 0L; - } else { - hasUpperBound = true; - upperDoubleBound = upperDouble; - } - } else { - hasUpperBound = false; - upperDoubleBound = 0L; - } - - if (matchesNothing) { - return DruidDoublePredicate.ALWAYS_FALSE; - } else { - return makeDoublePredicateFromBounds( - hasLowerBound, - hasUpperBound, - lowerStrict, - upperStrict, - lowerDoubleBound, - upperDoubleBound); - } - } - - private Supplier makeDoublePredicateSupplier() - { - Supplier doublePredicate = () -> createDruidDoublePredicate(); - return Suppliers.memoize(doublePredicate); - } - - private static DruidLongPredicate makeLongPredicateFromBounds( - final boolean hasLowerLongBound, - final boolean hasUpperLongBound, - final boolean lowerStrict, - final boolean upperStrict, - final long lowerLongBound, - final long upperLongBound) - { - if (hasLowerLongBound && hasUpperLongBound) { - if (upperStrict && lowerStrict) { - return input -> input > lowerLongBound && input < upperLongBound; - } else if (lowerStrict) { - return input -> input > lowerLongBound && input <= upperLongBound; - } else if (upperStrict) { - return input -> input >= lowerLongBound && input < upperLongBound; - } else { - return input -> input >= lowerLongBound && input <= upperLongBound; - } - } else if (hasUpperLongBound) { - if (upperStrict) { - return input -> input < upperLongBound; - } else { - return input -> input <= upperLongBound; - } - } else if (hasLowerLongBound) { - if (lowerStrict) { - return input -> input > lowerLongBound; - } else { - return input -> input >= lowerLongBound; - } - } else { - return DruidLongPredicate.ALWAYS_TRUE; - } - } - - private static DruidDoublePredicate makeDoublePredicateFromBounds( - final boolean hasLowerDoubleBound, - final boolean hasUpperDoubleBound, - final boolean lowerStrict, - final boolean upperStrict, - final double lowerDoubleBound, - final double upperDoubleBound) - { - if (hasLowerDoubleBound && hasUpperDoubleBound) { - if (upperStrict && lowerStrict) { - return input -> { - final int lowerComparing = Double.compare(input, lowerDoubleBound); - final int upperComparing = Double.compare(upperDoubleBound, input); - return ((lowerComparing > 0)) && (upperComparing > 0); - }; - } else if (lowerStrict) { - return input -> { - final int lowerComparing = Double.compare(input, lowerDoubleBound); - final int upperComparing = Double.compare(upperDoubleBound, input); - return (lowerComparing > 0) && (upperComparing >= 0); - }; - } else if (upperStrict) { - return input -> { - final int lowerComparing = Double.compare(input, lowerDoubleBound); - final int upperComparing = Double.compare(upperDoubleBound, input); - return (lowerComparing >= 0) && (upperComparing > 0); - }; - } else { - return input -> { - final int lowerComparing = Double.compare(input, lowerDoubleBound); - final int upperComparing = Double.compare(upperDoubleBound, input); - return (lowerComparing >= 0) && (upperComparing >= 0); - }; - } - } else if (hasUpperDoubleBound) { - if (upperStrict) { - return input -> { - final int upperComparing = Double.compare(upperDoubleBound, input); - return upperComparing > 0; - }; - } else { - return input -> { - final int upperComparing = Double.compare(upperDoubleBound, input); - return upperComparing >= 0; - }; - } - } else if (hasLowerDoubleBound) { - if (lowerStrict) { - return input -> { - final int lowerComparing = Double.compare(input, lowerDoubleBound); - return lowerComparing > 0; - }; - } else { - return input -> { - final int lowerComparing = Double.compare(input, lowerDoubleBound); - return lowerComparing >= 0; - }; - } - } else { - return DruidDoublePredicate.ALWAYS_TRUE; - } - } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java index 60580881c392..4e4a3b10ec94 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java @@ -48,7 +48,10 @@ @JsonSubTypes.Type(name = "like", value = LikeDimFilter.class), @JsonSubTypes.Type(name = "expression", value = ExpressionDimFilter.class), @JsonSubTypes.Type(name = "true", value = TrueDimFilter.class), - @JsonSubTypes.Type(name = "false", value = FalseDimFilter.class) + @JsonSubTypes.Type(name = "false", value = FalseDimFilter.class), + @JsonSubTypes.Type(name = "null", value = NullFilter.class), + @JsonSubTypes.Type(name = "equals", value = EqualityFilter.class), + @JsonSubTypes.Type(name = "range", value = RangeFilter.class) }) public interface DimFilter extends Cacheable { diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java index 3fcd719e25f5..27a0581d4752 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilterUtils.java @@ -54,6 +54,10 @@ public class DimFilterUtils static final byte TRUE_CACHE_ID = 0xF; static final byte FALSE_CACHE_ID = 0x11; public static final byte BLOOM_DIM_FILTER_CACHE_ID = 0x10; + static final byte NULL_CACHE_ID = 0x12; + static final byte EQUALS_CACHE_ID = 0x13; + static final byte RANGE_CACHE_ID = 0x14; + public static final byte STRING_SEPARATOR = (byte) 0xFF; diff --git a/processing/src/main/java/org/apache/druid/query/filter/DruidFloatPredicate.java b/processing/src/main/java/org/apache/druid/query/filter/DruidFloatPredicate.java index f805d2b0d29f..6559ca79f6a2 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DruidFloatPredicate.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DruidFloatPredicate.java @@ -30,6 +30,8 @@ public interface DruidFloatPredicate { DruidFloatPredicate ALWAYS_FALSE = input -> false; + DruidFloatPredicate ALWAYS_TRUE = input -> true; + DruidFloatPredicate MATCH_NULL_ONLY = new DruidFloatPredicate() { @Override diff --git a/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java b/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java index aff4346c4350..929f3acbba54 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java @@ -21,6 +21,11 @@ import com.google.common.base.Predicate; import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode; +import org.apache.druid.java.util.common.UOE; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; + +import javax.annotation.Nullable; @SubclassesMustOverrideEqualsAndHashCode public interface DruidPredicateFactory @@ -33,6 +38,11 @@ public interface DruidPredicateFactory DruidDoublePredicate makeDoublePredicate(); + default Predicate makeArrayPredicate(@Nullable TypeSignature inputType) + { + throw new UOE("Predicate does not support ARRAY types"); + } + /** * Object predicate is currently only used by vectorized matchers for non-string object selectors. This currently * means it will be used only if we encounter COMPLEX types, but will also include array types once they are more diff --git a/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java b/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java new file mode 100644 index 000000000000..4fd9b4aef539 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java @@ -0,0 +1,581 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; +import com.google.common.collect.TreeRangeSet; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; +import org.apache.druid.segment.BaseDoubleColumnValueSelector; +import org.apache.druid.segment.BaseFloatColumnValueSelector; +import org.apache.druid.segment.BaseLongColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.ColumnProcessorFactory; +import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.ColumnSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.filter.DimensionPredicateFilter; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.filter.PredicateValueMatcherFactory; +import org.apache.druid.segment.filter.ValueMatchers; +import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; +import org.apache.druid.segment.index.semantic.ValueIndexes; +import org.apache.druid.segment.nested.StructuredData; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.Comparator; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class EqualityFilter extends AbstractOptimizableDimFilter implements Filter +{ + private final String column; + private final ColumnType matchValueType; + private final Object matchValue; + @Nullable + private final ExtractionFn extractionFn; + @Nullable + private final FilterTuning filterTuning; + private final DruidPredicateFactory predicateFactory; + + @JsonCreator + public EqualityFilter( + @JsonProperty("column") String column, + @JsonProperty("matchValueType") ColumnType matchValueType, + @JsonProperty("matchValue") Object matchValue, + @JsonProperty("extractionFn") @Nullable ExtractionFn extractionFn, + @JsonProperty("filterTuning") @Nullable FilterTuning filterTuning + ) + { + if (column == null) { + throw InvalidInput.exception("Invalid equality filter, column cannot be null"); + } + this.column = column; + if (matchValueType == null) { + throw InvalidInput.exception("Invalid equality filter on column [%s], matchValueType cannot be null", column); + } + this.matchValueType = matchValueType; + if (matchValue == null) { + throw InvalidInput.exception("Invalid equality filter on column [%s], matchValue cannot be null", column); + } + this.matchValue = matchValue; + // remove once SQL planner no longer uses extractionFn + this.extractionFn = extractionFn; + this.filterTuning = filterTuning; + this.predicateFactory = new EqualityPredicateFactory(matchValue, matchValueType); + } + + @Override + public byte[] getCacheKey() + { + final TypeStrategy typeStrategy = matchValueType.getStrategy(); + final int size = typeStrategy.estimateSizeBytes(matchValue); + final ByteBuffer valueBuffer = ByteBuffer.allocate(size); + typeStrategy.write(valueBuffer, matchValue, size); + return new CacheKeyBuilder(DimFilterUtils.EQUALS_CACHE_ID) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendString(column) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendString(matchValueType.asTypeString()) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByteArray(valueBuffer.array()) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByteArray(extractionFn == null ? new byte[0] : extractionFn.getCacheKey()) + .build(); + } + + @Override + public DimFilter optimize() + { + return this; + } + + @Override + public Filter toFilter() + { + if (extractionFn == null) { + return this; + } else { + return new DimensionPredicateFilter(column, predicateFactory, extractionFn, filterTuning); + } + } + + @JsonProperty + public String getColumn() + { + return column; + } + + @JsonProperty + public ColumnType getMatchValueType() + { + return matchValueType; + } + + @JsonProperty + public Object getMatchValue() + { + return matchValue; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public ExtractionFn getExtractionFn() + { + return extractionFn; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public FilterTuning getFilterTuning() + { + return filterTuning; + } + + @Override + public String toString() + { + DimFilter.DimFilterToStringBuilder bob = + new DimFilter.DimFilterToStringBuilder().appendDimension(column, extractionFn) + .append(" = ") + .append(matchValue); + + if (!ColumnType.STRING.equals(matchValueType)) { + bob.append(" (" + matchValueType.asTypeString() + ")"); + } + return bob.appendFilterTuning(filterTuning).build(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EqualityFilter that = (EqualityFilter) o; + if (!column.equals(that.column)) { + return false; + } + if (!Objects.equals(matchValueType, that.matchValueType)) { + return false; + } + if (!Objects.equals(extractionFn, that.extractionFn)) { + return false; + } + if (!Objects.equals(filterTuning, that.filterTuning)) { + return false; + } + if (matchValueType.isArray()) { + // just use predicate to see if the values are the same + final ExprEval thatValue = ExprEval.ofType( + ExpressionType.fromColumnType(that.matchValueType), + that.matchValue + ); + final Predicate arrayPredicate = predicateFactory.makeArrayPredicate(matchValueType); + return arrayPredicate.apply(thatValue.asArray()); + } else { + return Objects.equals(matchValue, that.matchValue); + } + } + + @Override + public int hashCode() + { + return Objects.hash(column, matchValueType, matchValue, extractionFn, filterTuning); + } + + @Override + public RangeSet getDimensionRangeSet(String dimension) + { + if (!Objects.equals(getColumn(), dimension) || getExtractionFn() != null) { + return null; + } + RangeSet retSet = TreeRangeSet.create(); + retSet.add(Range.singleton(String.valueOf(matchValue))); + return retSet; + } + + @Nullable + @Override + public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) + { + if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) { + return null; + } + + final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column); + if (indexSupplier == null) { + return new AllFalseBitmapColumnIndex(selector); + } + + final ValueIndexes valueIndexes = indexSupplier.as(ValueIndexes.class); + if (valueIndexes != null) { + return valueIndexes.forValue(matchValue, matchValueType); + } + + if (matchValueType.isPrimitive()) { + final StringValueSetIndexes stringValueSetIndexes = indexSupplier.as(StringValueSetIndexes.class); + if (stringValueSetIndexes != null) { + + return stringValueSetIndexes.forValue(String.valueOf(matchValue)); + } + } + // column exists, but has no indexes we can use + return null; + } + + @Override + public ValueMatcher makeMatcher(ColumnSelectorFactory factory) + { + return ColumnProcessors.makeProcessor( + column, + new TypedConstantValueMatcherFactory(matchValue, matchValueType, predicateFactory), + factory + ); + } + + @Override + public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) + { + final ColumnCapabilities capabilities = factory.getColumnCapabilities(column); + + if (matchValueType.isPrimitive() && (capabilities == null || capabilities.isPrimitive())) { + return ColumnProcessors.makeVectorProcessor( + column, + VectorValueMatcherColumnProcessorFactory.instance(), + factory + ).makeMatcher(matchValue, matchValueType); + } + return ColumnProcessors.makeVectorProcessor( + column, + VectorValueMatcherColumnProcessorFactory.instance(), + factory + ).makeMatcher(new EqualityPredicateFactory(matchValue, matchValueType)); + } + + @Override + public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector) + { + return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector); + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return true; + } + + @Override + public Set getRequiredColumns() + { + return ImmutableSet.of(column); + } + + @Override + public boolean supportsRequiredColumnRewrite() + { + return true; + } + + @Override + public Filter rewriteRequiredColumns(Map columnRewrites) + { + String rewriteDimensionTo = columnRewrites.get(column); + + if (rewriteDimensionTo == null) { + throw new IAE( + "Received a non-applicable rewrite: %s, filter's dimension: %s", + columnRewrites, + columnRewrites + ); + } + + return new EqualityFilter( + rewriteDimensionTo, + matchValueType, + matchValue, + extractionFn, + filterTuning + ); + } + + private static class EqualityPredicateFactory implements DruidPredicateFactory + { + private final ExprEval matchValue; + private final ColumnType matchValueType; + private final Supplier> stringPredicateSupplier; + private final Supplier longPredicateSupplier; + private final Supplier floatPredicateSupplier; + private final Supplier doublePredicateSupplier; + private final ConcurrentHashMap, Predicate> arrayPredicates; + private final Supplier> typeDetectingArrayPredicateSupplier; + private final Supplier> objectPredicateSupplier; + + public EqualityPredicateFactory(Object matchValue, ColumnType matchValueType) + { + this.matchValue = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + this.matchValueType = matchValueType; + this.stringPredicateSupplier = makeStringPredicateSupplier(); + this.longPredicateSupplier = makeLongPredicateSupplier(); + this.floatPredicateSupplier = makeFloatPredicateSupplier(); + this.doublePredicateSupplier = makeDoublePredicateSupplier(); + this.objectPredicateSupplier = makeObjectPredicateSupplier(); + this.arrayPredicates = new ConcurrentHashMap<>(); + this.typeDetectingArrayPredicateSupplier = makeTypeDetectingArrayPredicate(); + } + + @Override + public Predicate makeStringPredicate() + { + return stringPredicateSupplier.get(); + } + + @Override + public DruidLongPredicate makeLongPredicate() + { + return longPredicateSupplier.get(); + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return floatPredicateSupplier.get(); + } + + @Override + public DruidDoublePredicate makeDoublePredicate() + { + return doublePredicateSupplier.get(); + } + + @Override + public Predicate makeArrayPredicate(@Nullable TypeSignature arrayType) + { + if (arrayType == null) { + // fall back to per row detection if input array type is unknown + return typeDetectingArrayPredicateSupplier.get(); + } + + return arrayPredicates.computeIfAbsent(arrayType, (existing) -> makeArrayPredicateInternal(arrayType)); + } + + @Override + public Predicate makeObjectPredicate() + { + return objectPredicateSupplier.get(); + } + + private Supplier> makeStringPredicateSupplier() + { + return Suppliers.memoize(() -> Predicates.equalTo(matchValue.castTo(ExpressionType.STRING).asString())); + } + + private Supplier makeLongPredicateSupplier() + { + return Suppliers.memoize(() -> { + final Long valueAsLong = (Long) matchValue.castTo(ExpressionType.LONG).valueOrDefault(); + + if (valueAsLong == null) { + return DruidLongPredicate.ALWAYS_FALSE; + } else { + // store the primitive, so we don't unbox for every comparison + final long unboxedLong = valueAsLong; + return input -> input == unboxedLong; + } + }); + } + + private Supplier makeFloatPredicateSupplier() + { + return Suppliers.memoize(() -> { + final Double doubleValue = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault(); + + if (doubleValue == null) { + return DruidFloatPredicate.ALWAYS_FALSE; + } else { + // Compare with floatToIntBits instead of == to canonicalize NaNs. + final int floatBits = Float.floatToIntBits(doubleValue.floatValue()); + return input -> Float.floatToIntBits(input) == floatBits; + } + }); + } + + private Supplier makeDoublePredicateSupplier() + { + return Suppliers.memoize(() -> { + final Double aDouble = (Double) matchValue.castTo(ExpressionType.DOUBLE).valueOrDefault(); + + if (aDouble == null) { + return DruidDoublePredicate.ALWAYS_FALSE; + } else { + // Compare with doubleToLongBits instead of == to canonicalize NaNs. + final long bits = Double.doubleToLongBits(aDouble); + return input -> Double.doubleToLongBits(input) == bits; + } + }); + } + + private Supplier> makeObjectPredicateSupplier() + { + return Suppliers.memoize(() -> { + if (matchValueType.equals(ColumnType.NESTED_DATA)) { + return input -> Objects.equals(StructuredData.unwrap(input), StructuredData.unwrap(matchValue.value())); + } + return Predicates.equalTo(matchValue.valueOrDefault()); + }); + } + + private Supplier> makeTypeDetectingArrayPredicate() + { + return Suppliers.memoize(() -> input -> { + final ExprEval eval = ExprEval.bestEffortOf(input); + final Comparator arrayComparator = eval.type().getNullableStrategy(); + final Object[] matchArray = matchValue.castTo(eval.type()).asArray(); + return arrayComparator.compare(input, matchArray) == 0; + }); + } + private Predicate makeArrayPredicateInternal(TypeSignature arrayType) + { + final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(arrayType); + final Comparator arrayComparator = arrayType.getNullableStrategy(); + final Object[] matchArray = matchValue.castTo(expressionType).asArray(); + return input -> arrayComparator.compare(input, matchArray) == 0; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + EqualityPredicateFactory that = (EqualityPredicateFactory) o; + return Objects.equals(matchValue, that.matchValue) && Objects.equals(matchValueType, that.matchValueType); + } + + + @Override + public int hashCode() + { + return Objects.hash(matchValue, matchValueType); + } + } + + private static class TypedConstantValueMatcherFactory implements ColumnProcessorFactory + { + private final ExprEval matchValue; + private final PredicateValueMatcherFactory predicateMatcherFactory; + + public TypedConstantValueMatcherFactory( + Object matchValue, + ColumnType matchValueType, + DruidPredicateFactory predicateFactory + ) + { + this.matchValue = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); + this.predicateMatcherFactory = new PredicateValueMatcherFactory(predicateFactory); + } + + @Override + public ColumnType defaultType() + { + return ColumnType.UNKNOWN_COMPLEX; + } + + @Override + public ValueMatcher makeDimensionProcessor(DimensionSelector selector, boolean multiValue) + { + return ValueMatchers.makeStringValueMatcher( + selector, + matchValue.castTo(ExpressionType.STRING).asString(), + multiValue + ); + } + + @Override + public ValueMatcher makeFloatProcessor(BaseFloatColumnValueSelector selector) + { + return ValueMatchers.makeFloatValueMatcher(selector, (float) matchValue.castTo(ExpressionType.DOUBLE).asDouble()); + } + + @Override + public ValueMatcher makeDoubleProcessor(BaseDoubleColumnValueSelector selector) + { + return ValueMatchers.makeDoubleValueMatcher(selector, matchValue.castTo(ExpressionType.DOUBLE).asDouble()); + } + + @Override + public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector) + { + return ValueMatchers.makeLongValueMatcher(selector, matchValue.castTo(ExpressionType.LONG).asLong()); + } + + @Override + public ValueMatcher makeArrayProcessor( + BaseObjectColumnValueSelector selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + return predicateMatcherFactory.makeArrayProcessor(selector, columnCapabilities); + } + + @Override + public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector selector) + { + return predicateMatcherFactory.makeComplexProcessor(selector); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/Filter.java b/processing/src/main/java/org/apache/druid/query/filter/Filter.java index 725db374ffb2..87d34e2e26b9 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/Filter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/Filter.java @@ -26,7 +26,7 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; diff --git a/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java index afddb0e42af4..47c3d78a237b 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/InDimFilter.java @@ -57,11 +57,11 @@ import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.StringValueSetIndex; -import org.apache.druid.segment.column.Utf8ValueSetIndex; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; +import org.apache.druid.segment.index.semantic.Utf8ValueSetIndexes; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -292,20 +292,20 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) if (indexSupplier == null) { // column doesn't exist, match against null - return Filters.makeNullIndex( + return Filters.makeMissingColumnNullIndex( predicateFactory.makeStringPredicate().apply(null), selector ); } - final Utf8ValueSetIndex utf8ValueSetIndex = indexSupplier.as(Utf8ValueSetIndex.class); - if (utf8ValueSetIndex != null) { - return utf8ValueSetIndex.forSortedValuesUtf8(valuesUtf8); + final Utf8ValueSetIndexes utf8ValueSetIndexes = indexSupplier.as(Utf8ValueSetIndexes.class); + if (utf8ValueSetIndexes != null) { + return utf8ValueSetIndexes.forSortedValuesUtf8(valuesUtf8); } - final StringValueSetIndex stringValueSetIndex = indexSupplier.as(StringValueSetIndex.class); - if (stringValueSetIndex != null) { - return stringValueSetIndex.forSortedValues(values); + final StringValueSetIndexes stringValueSetIndexes = indexSupplier.as(StringValueSetIndexes.class); + if (stringValueSetIndexes != null) { + return stringValueSetIndexes.forSortedValues(values); } } return Filters.makePredicateIndex( diff --git a/processing/src/main/java/org/apache/druid/query/filter/NullFilter.java b/processing/src/main/java/org/apache/druid/query/filter/NullFilter.java new file mode 100644 index 000000000000..0d88e3214194 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/NullFilter.java @@ -0,0 +1,316 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; +import com.google.common.collect.TreeRangeSet; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.ColumnSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.filter.DimensionPredicateFilter; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import javax.annotation.Nullable; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +public class NullFilter extends AbstractOptimizableDimFilter implements Filter +{ + public static NullFilter forColumn(String column) + { + return new NullFilter(column, null, null); + } + + private final String column; + @Nullable + private final ExtractionFn extractionFn; + @Nullable + private final FilterTuning filterTuning; + + @JsonCreator + public NullFilter( + @JsonProperty("column") String column, + @JsonProperty("extractionFn") @Nullable ExtractionFn extractionFn, + @JsonProperty("filterTuning") @Nullable FilterTuning filterTuning + ) + { + if (column == null) { + throw InvalidInput.exception("Invalid null filter, column cannot be null"); + } + this.column = column; + // remove once SQL planner no longer uses extractionFn + this.extractionFn = extractionFn; + this.filterTuning = filterTuning; + } + + @JsonProperty + public String getColumn() + { + return column; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public ExtractionFn getExtractionFn() + { + return extractionFn; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public FilterTuning getFilterTuning() + { + return filterTuning; + } + + @Override + public byte[] getCacheKey() + { + return new CacheKeyBuilder(DimFilterUtils.NULL_CACHE_ID) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendString(column) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByteArray(extractionFn == null ? new byte[0] : extractionFn.getCacheKey()) + .build(); + } + + @Override + public DimFilter optimize() + { + return this; + } + + @Override + public Filter toFilter() + { + if (extractionFn == null) { + return this; + } else { + return new DimensionPredicateFilter(column, NullPredicateFactory.INSTANCE, extractionFn, filterTuning); + } + } + + @Nullable + @Override + public RangeSet getDimensionRangeSet(String dimension) + { + RangeSet retSet = TreeRangeSet.create(); + // Nulls are less than empty String in segments + retSet.add(Range.lessThan("")); + return retSet; + } + + @Nullable + @Override + public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) + { + if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) { + return null; + } + final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column); + if (indexSupplier == null) { + return new AllTrueBitmapColumnIndex(selector); + } + final NullValueIndex nullValueIndex = indexSupplier.as(NullValueIndex.class); + if (nullValueIndex == null) { + return null; + } + return nullValueIndex.get(); + } + + @Override + public ValueMatcher makeMatcher(ColumnSelectorFactory factory) + { + return Filters.makeValueMatcher(factory, column, NullPredicateFactory.INSTANCE); + } + + @Override + public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) + { + return ColumnProcessors.makeVectorProcessor( + column, + VectorValueMatcherColumnProcessorFactory.instance(), + factory + ).makeMatcher(NullPredicateFactory.INSTANCE); + } + + @Override + public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector) + { + return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector); + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return true; + } + + @Override + public Set getRequiredColumns() + { + return ImmutableSet.of(column); + } + + @Override + public boolean supportsRequiredColumnRewrite() + { + return true; + } + + @Override + public Filter rewriteRequiredColumns(Map columnRewrites) + { + String rewriteDimensionTo = columnRewrites.get(column); + + if (rewriteDimensionTo == null) { + throw new IAE( + "Received a non-applicable rewrite: %s, filter's dimension: %s", + columnRewrites, + columnRewrites + ); + } + return new NullFilter(rewriteDimensionTo, extractionFn, filterTuning); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + NullFilter that = (NullFilter) o; + return Objects.equals(column, that.column) && + Objects.equals(extractionFn, that.extractionFn) && + Objects.equals(filterTuning, that.filterTuning); + } + + @Override + public int hashCode() + { + return Objects.hash(column, extractionFn, filterTuning); + } + + @Override + public String toString() + { + return new DimFilterToStringBuilder().appendDimension(column, extractionFn) + .append(" IS NULL") + .appendFilterTuning(filterTuning) + .build(); + } + + private static class NullPredicateFactory implements DruidPredicateFactory + { + public static final NullPredicateFactory INSTANCE = new NullPredicateFactory(); + + private NullPredicateFactory() + { + // no instantiation + } + + @Override + public Predicate makeStringPredicate() + { + return Predicates.isNull(); + } + + @Override + public DruidLongPredicate makeLongPredicate() + { + return DruidLongPredicate.MATCH_NULL_ONLY; + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + return DruidFloatPredicate.MATCH_NULL_ONLY; + } + + @Override + public DruidDoublePredicate makeDoublePredicate() + { + return DruidDoublePredicate.MATCH_NULL_ONLY; + } + + @Override + public Predicate makeArrayPredicate(@Nullable TypeSignature arrayType) + { + return Predicates.isNull(); + } + + @Override + public Predicate makeObjectPredicate() + { + return Predicates.isNull(); + } + + @Override + public int hashCode() + { + return super.hashCode(); + } + + @Override + public boolean equals(Object obj) + { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + return true; + } + + @Override + public String toString() + { + return "NullPredicateFactory{}"; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java b/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java new file mode 100644 index 000000000000..d9c1fe608bdd --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java @@ -0,0 +1,1062 @@ +/* + * 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; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.BoundType; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; +import com.google.common.collect.TreeRangeSet; +import org.apache.druid.error.InvalidInput; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.cache.CacheKeyBuilder; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; +import org.apache.druid.query.ordering.StringComparators; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.ColumnProcessors; +import org.apache.druid.segment.ColumnSelector; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.filter.DimensionPredicateFilter; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.NumericRangeIndexes; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class RangeFilter extends AbstractOptimizableDimFilter implements Filter +{ + private final String column; + private final ColumnType matchValueType; + private final ExpressionType matchValueExpressionType; + + @Nullable + private final Object upper; + @Nullable + private final Object lower; + private final ExprEval upperEval; + private final ExprEval lowerEval; + private final boolean lowerOpen; + private final boolean upperOpen; + @Nullable + private final ExtractionFn extractionFn; + @Nullable + private final FilterTuning filterTuning; + private final Supplier> stringPredicateSupplier; + private final Supplier longPredicateSupplier; + private final Supplier floatPredicateSupplier; + private final Supplier doublePredicateSupplier; + private final ConcurrentHashMap, Predicate> arrayPredicates; + private final Supplier> typeDetectingArrayPredicateSupplier; + + @JsonCreator + public RangeFilter( + @JsonProperty("column") String column, + @JsonProperty("matchValueType") ColumnType matchValueType, + @JsonProperty("lower") @Nullable Object lower, + @JsonProperty("upper") @Nullable Object upper, + @JsonProperty("lowerOpen") @Nullable Boolean lowerOpen, + @JsonProperty("upperOpen") @Nullable Boolean upperOpen, + @JsonProperty("extractionFn") @Nullable ExtractionFn extractionFn, + @JsonProperty("filterTuning") @Nullable FilterTuning filterTuning + ) + { + if (column == null) { + throw InvalidInput.exception("Invalid range filter, column cannot be null"); + } + this.column = column; + if (matchValueType == null) { + throw InvalidInput.exception("Invalid range filter on column [%s], matchValueType cannot be null", column); + } + this.matchValueType = matchValueType; + this.matchValueExpressionType = ExpressionType.fromColumnType(matchValueType); + this.upper = upper; + this.lower = lower; + this.upperEval = ExprEval.ofType(matchValueExpressionType, upper); + this.lowerEval = ExprEval.ofType(matchValueExpressionType, lower); + + if (lowerEval.value() == null && upperEval.value() == null) { + throw InvalidInput.exception( + "Invalid range filter on column [%s], lower and upper cannot be null at the same time", + column + ); + } + if (matchValueExpressionType.isNumeric()) { + if (lower != null && lowerEval.value() == null) { + throw InvalidInput.exception( + "Invalid range filter on column [%s], lower bound [%s] cannot be parsed as specified match value type [%s]", + column, + lower, + matchValueExpressionType + ); + } + if (upper != null && upperEval.value() == null) { + throw InvalidInput.exception( + "Invalid range filter on column [%s], upper bound [%s] cannot be parsed as specified match value type [%s]", + column, + upper, + matchValueExpressionType + ); + } + } + this.lowerOpen = lowerOpen != null && lowerOpen; + this.upperOpen = upperOpen != null && upperOpen; + // remove once SQL planner no longer uses extractionFn + this.extractionFn = extractionFn; + this.filterTuning = filterTuning; + this.stringPredicateSupplier = makeStringPredicateSupplier(); + this.longPredicateSupplier = makeLongPredicateSupplier(); + this.floatPredicateSupplier = makeFloatPredicateSupplier(); + this.doublePredicateSupplier = makeDoublePredicateSupplier(); + this.arrayPredicates = new ConcurrentHashMap<>(); + this.typeDetectingArrayPredicateSupplier = makeTypeDetectingArrayPredicate(); + + } + + @JsonProperty + public String getColumn() + { + return column; + } + + @JsonProperty + public ColumnType getMatchValueType() + { + return matchValueType; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Object getUpper() + { + return upper; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public Object getLower() + { + return lower; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean isLowerOpen() + { + return lowerOpen; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_DEFAULT) + public boolean isUpperOpen() + { + return upperOpen; + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public ExtractionFn getExtractionFn() + { + return extractionFn; + } + + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + @JsonProperty + public FilterTuning getFilterTuning() + { + return filterTuning; + } + + public boolean hasLowerBound() + { + return lower != null; + } + + public boolean hasUpperBound() + { + return upper != null; + } + + @Override + public byte[] getCacheKey() + { + final byte[] lowerBytes; + final byte[] upperBytes; + if (hasLowerBound()) { + final TypeStrategy typeStrategy = matchValueType.getStrategy(); + final int size = typeStrategy.estimateSizeBytes(lower); + final ByteBuffer valueBuffer = ByteBuffer.allocate(size); + typeStrategy.write(valueBuffer, lower, size); + lowerBytes = valueBuffer.array(); + } else { + lowerBytes = new byte[0]; + } + if (hasUpperBound()) { + final TypeStrategy typeStrategy = matchValueType.getStrategy(); + final int size = typeStrategy.estimateSizeBytes(upper); + final ByteBuffer valueBuffer = ByteBuffer.allocate(size); + typeStrategy.write(valueBuffer, upper, size); + upperBytes = valueBuffer.array(); + } else { + upperBytes = new byte[0]; + } + byte boundType = 0x1; + if (this.getLower() == null) { + boundType = 0x2; + } else if (this.getUpper() == null) { + boundType = 0x3; + } + + final byte lowerStrictByte = this.isLowerOpen() ? (byte) 1 : 0x0; + final byte upperStrictByte = this.isUpperOpen() ? (byte) 1 : 0x0; + + return new CacheKeyBuilder(DimFilterUtils.RANGE_CACHE_ID) + .appendByte(boundType) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendString(column) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendString(matchValueType.asTypeString()) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByteArray(upperBytes) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByteArray(lowerBytes) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByte(lowerStrictByte) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByte(upperStrictByte) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByteArray(extractionFn == null ? new byte[0] : extractionFn.getCacheKey()) + .build(); + } + + @Override + public DimFilter optimize() + { + return this; + } + + @Override + public Filter toFilter() + { + if (extractionFn != null) { + return new DimensionPredicateFilter(column, getPredicateFactory(), extractionFn, filterTuning); + } + return this; + } + + @Override + public RangeSet getDimensionRangeSet(String dimension) + { + if (!(Objects.equals(column, dimension) && getExtractionFn() == null)) { + return null; + } + + // We need to return a RangeSet, but we have Object, not String. We align with the interface by + // converting things to String, but we'd probably be better off adjusting the interface to something that is + // more type aware in the future + + String lowerString = lowerEval.asString(); + String upperString = upperEval.asString(); + RangeSet retSet = TreeRangeSet.create(); + Range range; + if (getLower() == null) { + range = isUpperOpen() ? Range.lessThan(upperString) : Range.atMost(upperString); + } else if (getUpper() == null) { + range = isLowerOpen() ? Range.greaterThan(lowerString) : Range.atLeast(lowerString); + } else { + range = Range.range( + lowerString, + isLowerOpen() ? BoundType.OPEN : BoundType.CLOSED, + upperString, + isUpperOpen() ? BoundType.OPEN : BoundType.CLOSED + ); + } + retSet.add(range); + return retSet; + } + + @Nullable + @Override + public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) + { + if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) { + return null; + } + final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column); + if (indexSupplier == null) { + return new AllFalseBitmapColumnIndex(selector); + } + + if (matchValueType.is(ValueType.STRING)) { + final LexicographicalRangeIndexes rangeIndexes = indexSupplier.as(LexicographicalRangeIndexes.class); + if (rangeIndexes != null) { + final String lower = hasLowerBound() ? lowerEval.asString() : null; + final String upper = hasUpperBound() ? upperEval.asString() : null; + return rangeIndexes.forRange(lower, lowerOpen, upper, upperOpen); + } + } + if (matchValueType.isNumeric()) { + final NumericRangeIndexes rangeIndexes = indexSupplier.as(NumericRangeIndexes.class); + if (rangeIndexes != null) { + final Number lower = (Number) lowerEval.value(); + final Number upper = (Number) upperEval.value(); + return rangeIndexes.forRange(lower, lowerOpen, upper, upperOpen); + } + } + + // fall back to predicate based index if it is available + final DruidPredicateIndexes predicateIndexes = indexSupplier.as(DruidPredicateIndexes.class); + if (predicateIndexes != null) { + return predicateIndexes.forPredicate(getPredicateFactory()); + } + // index doesn't exist + return null; + } + + @Override + public ValueMatcher makeMatcher(ColumnSelectorFactory factory) + { + return Filters.makeValueMatcher(factory, column, getPredicateFactory()); + } + + @Override + public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) + { + return ColumnProcessors.makeVectorProcessor( + column, + VectorValueMatcherColumnProcessorFactory.instance(), + factory + ).makeMatcher(getPredicateFactory()); + } + + @Override + public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector) + { + return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector); + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return true; + } + + @Override + public Set getRequiredColumns() + { + return ImmutableSet.of(column); + } + + @Override + public boolean supportsRequiredColumnRewrite() + { + return true; + } + + @Override + public Filter rewriteRequiredColumns(Map columnRewrites) + { + String rewriteDimensionTo = columnRewrites.get(column); + + if (rewriteDimensionTo == null) { + throw new IAE( + "Received a non-applicable rewrite: %s, filter's dimension: %s", + columnRewrites, + column + ); + } + return new RangeFilter( + rewriteDimensionTo, + matchValueType, + lower, + upper, + lowerOpen, + upperOpen, + extractionFn, + filterTuning + ); + } + + public boolean isEquality() + { + if (!hasUpperBound() || !hasLowerBound() || lowerOpen || upperOpen) { + return false; + } + if (matchValueType.isArray()) { + return Arrays.deepEquals( + lowerEval.asArray(), + upperEval.asArray() + ); + } else { + return Objects.equals(upper, lower); + } + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + RangeFilter that = (RangeFilter) o; + boolean upperSame; + boolean lowerSame; + if (matchValueType.isArray()) { + upperSame = Arrays.deepEquals( + upperEval.asArray(), + that.upperEval.asArray() + ); + lowerSame = Arrays.deepEquals( + lowerEval.asArray(), + that.lowerEval.asArray() + ); + } else { + upperSame = Objects.equals(upper, that.upper); + lowerSame = Objects.equals(lower, that.lower); + } + + return lowerOpen == that.lowerOpen && + upperOpen == that.upperOpen && + column.equals(that.column) && + Objects.equals(matchValueType, that.matchValueType) && + upperSame && + lowerSame && + Objects.equals(extractionFn, that.extractionFn) && + Objects.equals(filterTuning, that.filterTuning); + } + + @Override + public int hashCode() + { + return Objects.hash( + column, + matchValueType, + upper, + lower, + lowerOpen, + upperOpen, + extractionFn, + filterTuning + ); + } + + @Override + public String toString() + { + final DimFilterToStringBuilder builder = new DimFilterToStringBuilder(); + + if (lower != null) { + builder.append(lower); + if (lowerOpen) { + builder.append(" < "); + } else { + builder.append(" <= "); + } + } + + builder.appendDimension(column, extractionFn); + + builder.append(StringUtils.format(" as %s", matchValueType.toString())); + + if (upper != null) { + if (upperOpen) { + builder.append(" < "); + } else { + builder.append(" <= "); + } + builder.append(upper); + } + + return builder.appendFilterTuning(filterTuning).build(); + } + + private DruidPredicateFactory getPredicateFactory() + { + return new RangePredicateFactory(this); + } + + private Supplier makeLongPredicateSupplier() + { + return Suppliers.memoize(() -> { + final boolean hasLowerBound; + final boolean hasUpperBound; + final long lowerBound; + final long upperBound; + + if (hasLowerBound()) { + ExprEval lowerCast = lowerEval.castTo(ExpressionType.LONG); + if (lowerCast.isNumericNull()) { + hasLowerBound = false; + lowerBound = Long.MIN_VALUE; + } else { + lowerBound = lowerCast.asLong(); + hasLowerBound = true; + } + } else { + hasLowerBound = false; + lowerBound = Long.MIN_VALUE; + } + + if (hasUpperBound()) { + ExprEval upperCast = upperEval.castTo(ExpressionType.LONG); + if (upperCast.isNumericNull()) { + // upper value is not null, but isn't convertible to a long so is effectively null, nothing matches + return DruidLongPredicate.ALWAYS_FALSE; + } else { + hasUpperBound = true; + upperBound = upperCast.asLong(); + } + } else { + hasUpperBound = false; + upperBound = Long.MAX_VALUE; + } + final RangeType rangeType = RangeType.of(hasLowerBound, lowerOpen, hasUpperBound, upperOpen); + return makeLongPredicate(rangeType, lowerBound, upperBound); + }); + } + + private Supplier makeFloatPredicateSupplier() + { + return Suppliers.memoize(() -> { + DruidDoublePredicate doublePredicate = makeDoublePredicateSupplier().get(); + return doublePredicate::applyDouble; + }); + } + + private Supplier makeDoublePredicateSupplier() + { + return Suppliers.memoize(() -> { + final boolean hasLowerBound; + final boolean hasUpperBound; + final double lowerBound; + final double upperBound; + + if (hasLowerBound()) { + ExprEval lowerCast = lowerEval.castTo(ExpressionType.DOUBLE); + if (lowerCast.isNumericNull()) { + hasLowerBound = false; + lowerBound = Double.NEGATIVE_INFINITY; + } else { + lowerBound = lowerCast.asDouble(); + hasLowerBound = true; + } + } else { + hasLowerBound = false; + lowerBound = Double.NEGATIVE_INFINITY; + } + + if (hasUpperBound()) { + ExprEval upperCast = upperEval.castTo(ExpressionType.DOUBLE); + if (upperCast.isNumericNull()) { + // upper value is not null, but isn't convertible to a long so is effectively null, nothing matches + return DruidDoublePredicate.ALWAYS_FALSE; + } else { + hasUpperBound = true; + upperBound = upperCast.asDouble(); + } + } else { + hasUpperBound = false; + upperBound = Double.POSITIVE_INFINITY; + } + + RangeType rangeType = RangeType.of(hasLowerBound, lowerOpen, hasUpperBound, upperOpen); + return makeDoublePredicate(rangeType, lowerBound, upperBound); + }); + } + + private Supplier> makeStringPredicateSupplier() + { + return Suppliers.memoize(() -> { + final Comparator stringComparator = matchValueType.isNumeric() + ? StringComparators.NUMERIC + : StringComparators.LEXICOGRAPHIC; + final String lowerBound = hasLowerBound() ? lowerEval.castTo(ExpressionType.STRING).asString() : null; + final String upperBound = hasUpperBound() ? upperEval.castTo(ExpressionType.STRING).asString() : null; + + final RangeType rangeType = RangeType.of(hasLowerBound(), lowerOpen, hasUpperBound(), upperOpen); + + return makeComparatorPredicate(rangeType, stringComparator, lowerBound, upperBound); + }); + } + + + + private Predicate makeArrayPredicate(TypeSignature inputType) + { + final Comparator arrayComparator = inputType.getNullableStrategy(); + final ExpressionType expressionType = ExpressionType.fromColumnTypeStrict(inputType); + final RangeType rangeType = RangeType.of(hasLowerBound(), lowerOpen, hasUpperBound(), upperOpen); + final Object[] lowerBound = hasLowerBound() ? lowerEval.castTo(expressionType).asArray() : null; + final Object[] upperBound = hasUpperBound() ? upperEval.castTo(expressionType).asArray() : null; + return makeComparatorPredicate(rangeType, arrayComparator, lowerBound, upperBound); + } + + private Supplier> makeTypeDetectingArrayPredicate() + { + return Suppliers.memoize(() -> { + RangeType rangeType = RangeType.of(hasLowerBound(), lowerOpen, hasUpperBound(), upperOpen); + switch (rangeType) { + case OPEN: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); + final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Comparator comparator = val.type().getNullableStrategy(); + final int lowerComparing = comparator.compare(val.asArray(), lowerBound); + final int upperComparing = comparator.compare(upperBound, val.asArray()); + return ((lowerComparing > 0)) && (upperComparing > 0); + }; + case LOWER_OPEN_UPPER_CLOSED: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); + final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Comparator arrayComparator = val.type().getNullableStrategy(); + final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound); + final int upperComparing = arrayComparator.compare(upperBound, val.asArray()); + return (lowerComparing > 0) && (upperComparing >= 0); + }; + case LOWER_CLOSED_UPPER_OPEN: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); + final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Comparator arrayComparator = val.type().getNullableStrategy(); + final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound); + final int upperComparing = arrayComparator.compare(upperBound, val.asArray()); + return (lowerComparing >= 0) && (upperComparing > 0); + }; + case CLOSED: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); + final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Comparator arrayComparator = val.type().getNullableStrategy(); + final int lowerComparing = arrayComparator.compare(val.asArray(), lowerBound); + final int upperComparing = arrayComparator.compare(upperBound, val.asArray()); + return (lowerComparing >= 0) && (upperComparing >= 0); + }; + case LOWER_UNBOUNDED_UPPER_OPEN: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Comparator arrayComparator = val.type().getNullableStrategy(); + final int upperComparing = arrayComparator.compare(upperBound, val.asArray()); + return upperComparing > 0; + }; + case LOWER_UNBOUNDED_UPPER_CLOSED: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] upperBound = upperEval.castTo(val.type()).asArray(); + final Comparator arrayComparator = val.type().getNullableStrategy(); + final int upperComparing = arrayComparator.compare(upperBound, val.asArray()); + return upperComparing >= 0; + }; + case LOWER_OPEN_UPPER_UNBOUNDED: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); + final Comparator arrayComparator = val.type().getNullableStrategy(); + final int lowerComparing = arrayComparator.compare(lowerBound, val.asArray()); + return lowerComparing > 0; + }; + case LOWER_CLOSED_UPPER_UNBOUNDED: + return input -> { + if (input == null) { + return false; + } + ExprEval val = ExprEval.bestEffortOf(input); + final Object[] lowerBound = lowerEval.castTo(val.type()).asArray(); + final Comparator arrayComparator = val.type().getNullableStrategy(); + final int lowerComparing = arrayComparator.compare(lowerBound, val.asArray()); + return lowerComparing >= 0; + }; + case UNBOUNDED: + default: + return Predicates.notNull(); + } + }); + } + + private class RangePredicateFactory implements DruidPredicateFactory + { + private final RangeFilter rangeFilter; + + private RangePredicateFactory(RangeFilter rangeFilter) + { + this.rangeFilter = rangeFilter; + } + + @Override + public Predicate makeStringPredicate() + { + return stringPredicateSupplier.get(); + } + + @Override + public DruidLongPredicate makeLongPredicate() + { + if (matchValueType.isNumeric()) { + return longPredicateSupplier.get(); + } + Predicate stringPredicate = stringPredicateSupplier.get(); + return input -> stringPredicate.apply(String.valueOf(input)); + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + if (matchValueType.isNumeric()) { + return floatPredicateSupplier.get(); + } + Predicate stringPredicate = stringPredicateSupplier.get(); + return input -> stringPredicate.apply(String.valueOf(input)); + } + + @Override + public DruidDoublePredicate makeDoublePredicate() + { + if (matchValueType.isNumeric()) { + return doublePredicateSupplier.get(); + } + Predicate stringPredicate = stringPredicateSupplier.get(); + return input -> stringPredicate.apply(String.valueOf(input)); + } + + @Override + public Predicate makeArrayPredicate(@Nullable TypeSignature inputType) + { + if (inputType == null) { + return typeDetectingArrayPredicateSupplier.get(); + } + return arrayPredicates.computeIfAbsent( + inputType, + (existing) -> RangeFilter.this.makeArrayPredicate(inputType) + ); + } + + @Override + public int hashCode() + { + return rangeFilter.hashCode(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RangePredicateFactory that = (RangePredicateFactory) o; + return Objects.equals(rangeFilter, that.rangeFilter); + } + + @Override + public String toString() + { + return "RangePredicateFactory{" + + "rangeFilter=" + rangeFilter + + '}'; + } + } + + public static DruidLongPredicate makeLongPredicate( + final RangeType rangeType, + final long lowerLongBound, + final long upperLongBound + ) + { + switch (rangeType) { + case OPEN: + return input -> input > lowerLongBound && input < upperLongBound; + case LOWER_OPEN_UPPER_CLOSED: + return input -> input > lowerLongBound && input <= upperLongBound; + case LOWER_CLOSED_UPPER_OPEN: + return input -> input >= lowerLongBound && input < upperLongBound; + case CLOSED: + return input -> input >= lowerLongBound && input <= upperLongBound; + case LOWER_UNBOUNDED_UPPER_OPEN: + return input -> input < upperLongBound; + case LOWER_UNBOUNDED_UPPER_CLOSED: + return input -> input <= upperLongBound; + case LOWER_OPEN_UPPER_UNBOUNDED: + return input -> input > lowerLongBound; + case LOWER_CLOSED_UPPER_UNBOUNDED: + return input -> input >= lowerLongBound; + case UNBOUNDED: + default: + return DruidLongPredicate.ALWAYS_TRUE; + } + } + + public static DruidDoublePredicate makeDoublePredicate( + final RangeType rangeType, + final double lowerDoubleBound, + final double upperDoubleBound + ) + { + switch (rangeType) { + case OPEN: + return input -> { + final int lowerComparing = Double.compare(input, lowerDoubleBound); + final int upperComparing = Double.compare(upperDoubleBound, input); + return ((lowerComparing > 0)) && (upperComparing > 0); + }; + case LOWER_OPEN_UPPER_CLOSED: + return input -> { + final int lowerComparing = Double.compare(input, lowerDoubleBound); + final int upperComparing = Double.compare(upperDoubleBound, input); + return (lowerComparing > 0) && (upperComparing >= 0); + }; + case LOWER_CLOSED_UPPER_OPEN: + return input -> { + final int lowerComparing = Double.compare(input, lowerDoubleBound); + final int upperComparing = Double.compare(upperDoubleBound, input); + return (lowerComparing >= 0) && (upperComparing > 0); + }; + case CLOSED: + return input -> { + final int lowerComparing = Double.compare(input, lowerDoubleBound); + final int upperComparing = Double.compare(upperDoubleBound, input); + return (lowerComparing >= 0) && (upperComparing >= 0); + }; + case LOWER_UNBOUNDED_UPPER_OPEN: + return input -> { + final int upperComparing = Double.compare(upperDoubleBound, input); + return upperComparing > 0; + }; + case LOWER_UNBOUNDED_UPPER_CLOSED: + return input -> { + final int upperComparing = Double.compare(upperDoubleBound, input); + return upperComparing >= 0; + }; + case LOWER_OPEN_UPPER_UNBOUNDED: + return input -> { + final int lowerComparing = Double.compare(input, lowerDoubleBound); + return lowerComparing > 0; + }; + case LOWER_CLOSED_UPPER_UNBOUNDED: + return input -> { + final int lowerComparing = Double.compare(input, lowerDoubleBound); + return lowerComparing >= 0; + }; + case UNBOUNDED: + default: + return DruidDoublePredicate.ALWAYS_TRUE; + } + } + + public static Predicate makeComparatorPredicate( + RangeType rangeType, + Comparator comparator, + @Nullable T lowerBound, + @Nullable T upperBound + ) + { + switch (rangeType) { + case OPEN: + return input -> { + if (input == null) { + return false; + } + final int lowerComparing = comparator.compare(input, lowerBound); + final int upperComparing = comparator.compare(upperBound, input); + return ((lowerComparing > 0)) && (upperComparing > 0); + }; + case LOWER_OPEN_UPPER_CLOSED: + return input -> { + if (input == null) { + return false; + } + final int lowerComparing = comparator.compare(input, lowerBound); + final int upperComparing = comparator.compare(upperBound, input); + return (lowerComparing > 0) && (upperComparing >= 0); + }; + case LOWER_CLOSED_UPPER_OPEN: + return input -> { + if (input == null) { + return false; + } + final int lowerComparing = comparator.compare(input, lowerBound); + final int upperComparing = comparator.compare(upperBound, input); + return (lowerComparing >= 0) && (upperComparing > 0); + }; + case CLOSED: + return input -> { + if (input == null) { + return false; + } + final int lowerComparing = comparator.compare(input, lowerBound); + final int upperComparing = comparator.compare(upperBound, input); + return (lowerComparing >= 0) && (upperComparing >= 0); + }; + case LOWER_UNBOUNDED_UPPER_OPEN: + return input -> { + if (input == null) { + return false; + } + final int upperComparing = comparator.compare(upperBound, input); + return upperComparing > 0; + }; + case LOWER_UNBOUNDED_UPPER_CLOSED: + return input -> { + if (input == null) { + return false; + } + final int upperComparing = comparator.compare(upperBound, input); + return upperComparing >= 0; + }; + case LOWER_OPEN_UPPER_UNBOUNDED: + return input -> { + if (input == null) { + return false; + } + final int lowerComparing = comparator.compare(input, lowerBound); + return lowerComparing > 0; + }; + case LOWER_CLOSED_UPPER_UNBOUNDED: + return input -> { + if (input == null) { + return false; + } + final int lowerComparing = comparator.compare(input, lowerBound); + return lowerComparing >= 0; + }; + case UNBOUNDED: + default: + return Predicates.notNull(); + } + } + + public enum RangeType + { + /** + * (...) + */ + OPEN, + /** + * [...] + */ + CLOSED, + /** + * [...) + */ + LOWER_CLOSED_UPPER_OPEN, + /** + * (...] + */ + LOWER_OPEN_UPPER_CLOSED, + /** + * (...∞ + */ + LOWER_OPEN_UPPER_UNBOUNDED, + /** + * [...∞ + */ + LOWER_CLOSED_UPPER_UNBOUNDED, + /** + * -∞...) + */ + LOWER_UNBOUNDED_UPPER_OPEN, + /** + * -∞...] + */ + LOWER_UNBOUNDED_UPPER_CLOSED, + /** + * -∞...∞ + */ + UNBOUNDED; + + public static RangeType of(boolean hasLower, boolean lowerOpen, boolean hasUpper, boolean upperOpen) + { + if (hasLower && hasUpper) { + if (lowerOpen) { + return upperOpen ? OPEN : LOWER_OPEN_UPPER_CLOSED; + } else { + return upperOpen ? LOWER_CLOSED_UPPER_OPEN : CLOSED; + } + } else if (hasLower) { + return lowerOpen ? LOWER_OPEN_UPPER_UNBOUNDED : LOWER_CLOSED_UPPER_UNBOUNDED; + } else if (hasUpper) { + return upperOpen ? LOWER_UNBOUNDED_UPPER_OPEN : LOWER_UNBOUNDED_UPPER_CLOSED; + } + return UNBOUNDED; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java new file mode 100644 index 000000000000..13c53aeee023 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java @@ -0,0 +1,97 @@ +/* + * 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 com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.vector.VectorObjectSelector; + +import javax.annotation.Nullable; + +public class ArrayVectorValueMatcher implements VectorValueMatcherFactory +{ + protected final TypeSignature columnType; + protected final VectorObjectSelector selector; + + public ArrayVectorValueMatcher( + TypeSignature columnType, + VectorObjectSelector selector + ) + { + this.columnType = columnType; + this.selector = selector; + } + + @Override + public VectorValueMatcher makeMatcher(@Nullable String value) + { + throw new UnsupportedOperationException( + "Vectorized matcher cannot make string matcher for ARRAY types" + ); + } + + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + throw new UnsupportedOperationException( + "Vectorized matcher cannot make object matcher for ARRAY types" + ); + } + + @Override + public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory) + { + final Predicate predicate = predicateFactory.makeArrayPredicate(columnType); + + return new BaseVectorValueMatcher(selector) + { + final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); + + @Override + public ReadableVectorMatch match(final ReadableVectorMatch mask) + { + final Object[] vector = selector.getObjectVector(); + final int[] selection = match.getSelection(); + + int numRows = 0; + + for (int i = 0; i < mask.getSelectionSize(); i++) { + final int rowNum = mask.getSelection()[i]; + Object o = vector[rowNum]; + if (o == null || o instanceof Object[]) { + if (predicate.apply((Object[]) o)) { + selection[numRows++] = rowNum; + } + } else { + if (predicate.apply(new Object[]{o})) { + selection[numRows++] = rowNum; + } + } + } + + match.setSelectionSize(numRows); + return match; + } + }; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java index 1b95cb977bbb..66064fc4693c 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java @@ -19,9 +19,12 @@ package org.apache.druid.query.filter.vector; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DruidDoublePredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; @@ -48,8 +51,22 @@ public VectorValueMatcher makeMatcher(@Nullable final String value) return BooleanVectorValueMatcher.of(selector, false); } - final double matchValDouble = matchVal; + return makeDoubleMatcher(matchVal); + } + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); + ExprEval cast = eval.castTo(ExpressionType.DOUBLE); + if (cast.isNumericNull()) { + return makeNullValueMatcher(selector); + } + return makeDoubleMatcher(cast.asDouble()); + } + + private BaseVectorValueMatcher makeDoubleMatcher(double matchValDouble) + { return new BaseVectorValueMatcher(selector) { final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @@ -74,12 +91,12 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; } + @Override public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory) { @@ -111,7 +128,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java index cd62e406ab3b..ed8b787668eb 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java @@ -19,9 +19,12 @@ package org.apache.druid.query.filter.vector; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DruidFloatPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; @@ -50,6 +53,22 @@ public VectorValueMatcher makeMatcher(@Nullable final String value) final float matchValFloat = matchVal; + return makeFloatMatcher(matchValFloat); + } + + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); + ExprEval cast = eval.castTo(ExpressionType.DOUBLE); + if (cast.isNumericNull()) { + return makeNullValueMatcher(selector); + } + return makeFloatMatcher((float) cast.asDouble()); + } + + private BaseVectorValueMatcher makeFloatMatcher(float matchValFloat) + { return new BaseVectorValueMatcher(selector) { final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @@ -75,7 +94,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; @@ -112,7 +130,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java index 55084a67c275..a38703451c5c 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java @@ -19,9 +19,12 @@ package org.apache.druid.query.filter.vector; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DruidLongPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; @@ -50,6 +53,22 @@ public VectorValueMatcher makeMatcher(@Nullable final String value) final long matchValLong = matchVal; + return makeLongMatcher(matchValLong); + } + + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); + ExprEval cast = eval.castTo(ExpressionType.LONG); + if (cast.isNumericNull()) { + return makeNullValueMatcher(selector); + } + return makeLongMatcher(cast.asLong()); + } + + private BaseVectorValueMatcher makeLongMatcher(long matchValLong) + { return new BaseVectorValueMatcher(selector) { final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @@ -75,7 +94,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; @@ -112,7 +130,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java index d192673784dc..866a9608fdd9 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java @@ -21,8 +21,11 @@ import com.google.common.base.Predicate; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; @@ -89,7 +92,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; @@ -98,6 +100,14 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } } + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); + ExprEval cast = eval.castTo(ExpressionType.STRING); + return makeMatcher(cast.asString()); + } + @Override public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory) { @@ -159,7 +169,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; @@ -199,7 +208,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java index 05f30ce8fbef..42ca5eeb8190 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java @@ -21,6 +21,7 @@ import com.google.common.base.Predicate; import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorObjectSelector; import javax.annotation.Nullable; @@ -52,6 +53,12 @@ public VectorValueMatcher makeMatcher(@Nullable String value) return BooleanVectorValueMatcher.of(selector, value == null); } + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + return BooleanVectorValueMatcher.of(selector, value == null); + } + @Override public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory) { @@ -77,7 +84,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java index 49646fb97567..181241726f4f 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java @@ -21,8 +21,11 @@ import com.google.common.base.Predicate; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; @@ -97,7 +100,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; @@ -106,6 +108,14 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } } + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); + ExprEval cast = eval.castTo(ExpressionType.STRING); + return makeMatcher(cast.asString()); + } + @Override public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory) { @@ -157,7 +167,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; @@ -183,7 +192,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java index 6e5e09dbc253..e587b313cae2 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java @@ -20,7 +20,10 @@ package org.apache.druid.query.filter.vector; import com.google.common.base.Predicate; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorObjectSelector; import javax.annotation.Nullable; @@ -61,12 +64,19 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; } + @Override + public VectorValueMatcher makeMatcher(Object value, ColumnType type) + { + ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(type), value); + ExprEval cast = eval.castTo(ExpressionType.STRING); + return makeMatcher(cast.asString()); + } + @Override public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory) { @@ -92,7 +102,6 @@ public ReadableVectorMatch match(final ReadableVectorMatch mask) } match.setSelectionSize(numRows); - assert match.isValid(mask); return match; } }; diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java index 93826c729174..0d16ee24230b 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactory.java @@ -86,6 +86,12 @@ public VectorValueMatcherFactory makeLongProcessor( return new LongVectorValueMatcher(selector); } + @Override + public VectorValueMatcherFactory makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + return new ArrayVectorValueMatcher(capabilities, selector); + } + @Override public VectorValueMatcherFactory makeObjectProcessor( final ColumnCapabilities capabilities, diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java index e61373cc80fb..666e85f4f238 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java @@ -20,6 +20,7 @@ package org.apache.druid.query.filter.vector; import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; @@ -28,6 +29,8 @@ public interface VectorValueMatcherFactory { VectorValueMatcher makeMatcher(@Nullable String value); + VectorValueMatcher makeMatcher(Object value, ColumnType type); + VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory); default VectorValueMatcher makeNullValueMatcher(VectorValueSelector selector) diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java index 3af16ea3d0dc..9c73d0714087 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/vector/GroupByVectorColumnProcessorFactory.java @@ -106,6 +106,14 @@ public GroupByVectorColumnSelector makeLongProcessor( return new NullableLongGroupByVectorColumnSelector(selector); } + @Override + public GroupByVectorColumnSelector makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + throw new UnsupportedOperationException( + "Vectorized groupBys on ARRAY columns are not yet implemented" + ); + } + @Override public GroupByVectorColumnSelector makeObjectProcessor( final ColumnCapabilities capabilities, @@ -113,6 +121,11 @@ public GroupByVectorColumnSelector makeObjectProcessor( ) { if (capabilities.is(ValueType.STRING)) { + if (capabilities.hasMultipleValues().isTrue()) { + throw new UnsupportedOperationException( + "Vectorized groupBys on multi-value dictionary-encoded dimensions are not yet implemented" + ); + } return new DictionaryBuildingSingleValueStringGroupByVectorColumnSelector(selector); } return NilGroupByVectorColumnSelector.INSTANCE; 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 7511cedae217..81fcf34ed013 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 @@ -44,12 +44,12 @@ import org.apache.druid.segment.column.ColumnTypeFactory; import org.apache.druid.segment.column.ComplexColumn; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.serde.ComplexMetricSerde; import org.apache.druid.segment.serde.ComplexMetrics; import org.joda.time.DateTime; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java index a6c9642c3bac..bb011ff79312 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFramedOnHeapAggregatable.java @@ -526,7 +526,6 @@ public CumulativeColumnSelectorFactory(AggregatorFactory factory, Object[] resul .setDictionaryEncoded(false) .setHasMultipleValues(false) .setDictionaryValuesUnique(false) - .setFilterable(false) .setType(factory.getIntermediateType()); } diff --git a/processing/src/main/java/org/apache/druid/query/search/AutoStrategy.java b/processing/src/main/java/org/apache/druid/query/search/AutoStrategy.java index 36ddcc3ad9f7..29d5c3510040 100644 --- a/processing/src/main/java/org/apache/druid/query/search/AutoStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/AutoStrategy.java @@ -29,7 +29,7 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import java.util.List; diff --git a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java index 60d3b17ba6ce..1f2307a6ee0e 100644 --- a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java @@ -39,12 +39,12 @@ import org.apache.druid.segment.Segment; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.column.NumericColumn; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.virtual.VirtualizedColumnInspector; import org.joda.time.Interval; diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java index 3ce8b5f4ba62..f0b52669fa2a 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnIndexer.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.dimension.DimensionSpec; @@ -223,10 +224,13 @@ public DimensionSelector makeDimensionSelector( ) { final int dimIndex = desc.getIndex(); + if (fieldIndexers.size() == 0 && isConstant && !hasNestedData) { + return DimensionSelector.constant(null, spec.getExtractionFn()); + } final ColumnValueSelector rootLiteralSelector = getRootLiteralValueSelector(currEntry, dimIndex); if (rootLiteralSelector != null) { final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); - final ColumnType rootType = root.getTypes().getSingleType(); + final ColumnType rootType = root.isSingleType() ? root.getTypes().getSingleType() : getLogicalType(); if (rootType.isArray()) { throw new UOE( "makeDimensionSelector is not supported, column [%s] is [%s] typed and should only use makeColumnValueSelector", @@ -234,17 +238,31 @@ public DimensionSelector makeDimensionSelector( rootType ); } + if (spec.getExtractionFn() == null) { + return new BaseSingleValueDimensionSelector() + { + @Nullable + @Override + protected String getValue() + { + return Evals.asString(rootLiteralSelector.getObject()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + }; + } return new BaseSingleValueDimensionSelector() { @Nullable @Override protected String getValue() { - final Object o = rootLiteralSelector.getObject(); - if (o == null) { - return null; - } - return o.toString(); + final String s = Evals.asString(rootLiteralSelector.getObject()); + return spec.getExtractionFn().apply(s); } @Override @@ -457,14 +475,14 @@ private ColumnValueSelector getRootLiteralValueSelector( int dimIndex ) { - if (fieldIndexers.size() > 1) { + if (fieldIndexers.size() > 1 || hasNestedData) { return null; } final FieldIndexer root = fieldIndexers.get(NestedPathFinder.JSON_PATH_ROOT); - if (root == null || !root.isSingleType()) { + if (root == null) { return null; } - final Object defaultValue = getDefaultValueForType(root.getTypes().getSingleType()); + final Object defaultValue = getDefaultValueForType(getLogicalType()); return new ColumnValueSelector() { @Override diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java index 1c4ca2f6f723..1e243f275526 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnProcessorFactory.java @@ -20,8 +20,11 @@ package org.apache.druid.segment; import org.apache.druid.query.dimension.ColumnSelectorStrategyFactory; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; +import javax.annotation.Nullable; + /** * Class that encapsulates knowledge about how to create "column processors", which are... objects that process columns * and want to have type-specific logic. Used by {@link ColumnProcessors#makeProcessor}. @@ -76,6 +79,14 @@ public interface ColumnProcessorFactory */ T makeLongProcessor(BaseLongColumnValueSelector selector); + /** + * + * @param selector array selector + * @param columnCapabilities information about the underlying column to match. Null here just means the capabilities + * are unknown, and not necessarily indicative that the column doesn't exist + */ + T makeArrayProcessor(BaseObjectColumnValueSelector selector, @Nullable ColumnCapabilities columnCapabilities); + /** * Create a processor for a complex 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 28eaef4c2697..bf6c399519f4 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnProcessors.java @@ -286,6 +286,11 @@ private static T makeProcessorInternal( return processorFactory.makeFloatProcessor(valueSelectorFunction.apply(selectorFactory)); case DOUBLE: return processorFactory.makeDoubleProcessor(valueSelectorFunction.apply(selectorFactory)); + case ARRAY: + return processorFactory.makeArrayProcessor( + valueSelectorFunction.apply(selectorFactory), + capabilities + ); case COMPLEX: return processorFactory.makeComplexProcessor(valueSelectorFunction.apply(selectorFactory)); default: @@ -359,6 +364,8 @@ private static T makeVectorProcessorInternal( return processorFactory.makeFloatProcessor(capabilities, valueSelectorFn.apply(selectorFactory)); case DOUBLE: return processorFactory.makeDoubleProcessor(capabilities, valueSelectorFn.apply(selectorFactory)); + case ARRAY: + return processorFactory.makeArrayProcessor(capabilities, objectSelectorFn.apply(selectorFactory)); case COMPLEX: return processorFactory.makeObjectProcessor(capabilities, objectSelectorFn.apply(selectorFactory)); default: diff --git a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelector.java b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelector.java index 6b7e0b666b65..62d2870371b9 100644 --- a/processing/src/main/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelector.java @@ -80,10 +80,9 @@ public ColumnIndexSupplier getIndexSupplier(String column) indexSupplier = virtualColumns.getIndexSupplier(column, columnSelector); } else { final ColumnHolder columnHolder = columnSelector.getColumnHolder(column); - // for missing columns and columns with types that do not support filtering, - // treat the column as if it were full of nulls. This allows callers to fabricate an 'all true' or 'all false' + // for missing columns we return null here. This allows callers to fabricate an 'all true' or 'all false' // index so that filters which match the values can still use "indexes". - if (columnHolder == null || !columnHolder.getCapabilities().isFilterable()) { + if (columnHolder == null) { return null; } indexSupplier = columnHolder.getIndexSupplier(); diff --git a/processing/src/main/java/org/apache/druid/segment/FilterAnalysis.java b/processing/src/main/java/org/apache/druid/segment/FilterAnalysis.java index fc9529f7d29d..c700c5b5e86b 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilterAnalysis.java +++ b/processing/src/main/java/org/apache/druid/segment/FilterAnalysis.java @@ -24,9 +24,9 @@ import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.filter.AndFilter; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.index.BitmapColumnIndex; import javax.annotation.Nullable; import java.util.ArrayList; diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java index 64fd12d374a6..f6103c3ce2fa 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java @@ -27,10 +27,10 @@ import org.apache.druid.query.filter.RowOffsetMatcherFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.data.Offset; import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.roaringbitmap.IntIterator; public final class FilteredOffset extends Offset diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java index 9ad2e734dc7d..f97f4a1bade7 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexIndexableAdapter.java @@ -30,11 +30,11 @@ import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; import org.apache.druid.segment.data.BitmapValues; import org.apache.druid.segment.data.CloseableIndexed; import org.apache.druid.segment.data.ImmutableBitmapValues; import org.apache.druid.segment.data.IndexedIterable; +import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; import org.apache.druid.segment.nested.NestedCommonFormatColumn; import org.apache.druid.segment.nested.NestedDataComplexTypeSerde; import org.apache.druid.segment.nested.SortedValueDictionary; diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index c0c117b8c8a6..ead7900c6a80 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -33,9 +33,9 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.column.NumericColumn; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.vector.VectorCursor; import org.joda.time.DateTime; import org.joda.time.Interval; diff --git a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java index d9f4bbce456d..4df6a81ff96b 100644 --- a/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/VectorColumnProcessorFactory.java @@ -83,6 +83,8 @@ T makeMultiValueDimensionProcessor( */ T makeLongProcessor(ColumnCapabilities capabilities, VectorValueSelector selector); + T makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector); + /** * Called when {@link ColumnCapabilities#getType()} is COMPLEX. May also be called for STRING typed columns in * cases where the dictionary does not exist or is not expected to be useful. diff --git a/processing/src/main/java/org/apache/druid/segment/column/CapabilitiesBasedFormat.java b/processing/src/main/java/org/apache/druid/segment/column/CapabilitiesBasedFormat.java index d04123d46536..81da30cceede 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/CapabilitiesBasedFormat.java +++ b/processing/src/main/java/org/apache/druid/segment/column/CapabilitiesBasedFormat.java @@ -136,7 +136,6 @@ public ColumnFormat merge(@Nullable ColumnFormat otherFormat) if (merged.hasSpatialIndexes() != otherSnapshot.hasSpatialIndexes()) { merged.setHasSpatialIndexes(merged.hasSpatialIndexes() || otherSnapshot.hasSpatialIndexes()); } - merged.setFilterable(merged.isFilterable() && otherSnapshot.isFilterable()); return new CapabilitiesBasedFormat(merged); } 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 e6d5b418786b..f84873700943 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 @@ -96,13 +96,6 @@ public ColumnBuilder setDictionaryEncodedColumnSupplier(Supplier columnSupplier) { checkColumnSupplierNotSet(); 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 4ea27c495a32..93769376b245 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 @@ -74,12 +74,6 @@ public interface ColumnCapabilities extends TypeSignature */ boolean hasSpatialIndexes(); - /** - * All Druid primitive columns support filtering, maybe with or without indexes, but by default complex columns - * do not support direct filtering, unless provided by through a custom implementation. - */ - boolean isFilterable(); - /** * Does this column contain null values? If so, callers, especially for primitive numeric columns, will need to check * for null value rows and act accordingly 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 74d090be8c4b..f8464a0cf716 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 @@ -79,7 +79,6 @@ public static ColumnCapabilitiesImpl copyOf(@Nullable final ColumnCapabilities o capabilities.dictionaryValuesSorted = other.areDictionaryValuesSorted(); capabilities.dictionaryValuesUnique = other.areDictionaryValuesUnique(); capabilities.hasNulls = other.hasNulls(); - capabilities.filterable = other.isFilterable(); } return capabilities; } @@ -179,8 +178,6 @@ public static ColumnCapabilitiesImpl createSimpleArrayColumnCapabilities(TypeSig @JsonIgnore private Capable dictionaryValuesUnique = Capable.UNKNOWN; @JsonIgnore - private boolean filterable; - @JsonIgnore private Capable hasNulls = Capable.UNKNOWN; @Nullable @@ -314,16 +311,4 @@ public ColumnCapabilitiesImpl setHasNulls(Capable hasNulls) this.hasNulls = hasNulls; return this; } - - @Override - public boolean isFilterable() - { - return (type != null && (isPrimitive() || isArray())) || filterable; - } - - public ColumnCapabilitiesImpl setFilterable(boolean filterable) - { - this.filterable = filterable; - return this; - } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java index bad9dc6a6f38..ae5ca7ff8a30 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java @@ -19,9 +19,16 @@ package org.apache.druid.segment.column; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.NumericRangeIndexes; + public interface ColumnConfig { - ColumnConfig DEFAULT = new ColumnConfig() {}; + ColumnConfig DEFAULT = new ColumnConfig() + { + }; ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() { @@ -41,7 +48,7 @@ public double skipValuePredicateIndexScale() /** * If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap - * index operations required to perform to use a {@link LexicographicalRangeIndex} or {@link NumericRangeIndex}, + * index operations required to perform to use {@link LexicographicalRangeIndexes} or {@link NumericRangeIndexes}, * then for any {@link ColumnIndexSupplier} which chooses to participate in this config it will skip computing the * index, indicated by a return value of null from the 'forRange' methods, to force the filter to be processed * with a scan using a {@link org.apache.druid.query.filter.ValueMatcher} instead. @@ -74,7 +81,7 @@ default double skipValueRangeIndexScale() /** * If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap - * index operations required to perform to use a {@link DruidPredicateIndex} then for any {@link ColumnIndexSupplier} + * index operations required to perform to use {@link DruidPredicateIndexes} then for any {@link ColumnIndexSupplier} * which chooses to participate in this config it will skip computing the index, in favor of doing a full scan and * using a {@link org.apache.druid.query.filter.ValueMatcher} instead. This is indicated returning null from * {@link ColumnIndexSupplier#as(Class)} even though it would have otherwise been able to create a diff --git a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java index 270e3b0ff677..d411589d99ed 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java @@ -39,9 +39,9 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java index 00cc5898989b..d08160a3166c 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java @@ -44,12 +44,15 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.NumericRangeIndex; +import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.NumericRangeIndexes; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -77,36 +80,37 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) if (!Filters.checkFilterTuningUseIndex(boundDimFilter.getDimension(), selector, filterTuning)) { return null; } + final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(boundDimFilter.getDimension()); + if (indexSupplier == null) { + // missing column -> match all rows if the predicate matches null; match no rows otherwise + return getPredicateFactory().makeStringPredicate().apply(null) + ? new AllTrueBitmapColumnIndex(selector) + : new AllFalseBitmapColumnIndex(selector); + } + if (supportStringShortCircuit()) { - final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(boundDimFilter.getDimension()); - if (indexSupplier == null) { - return Filters.makeNullIndex(doesMatchNull(), selector); - } - final LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); - if (rangeIndex != null) { - final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange( + final LexicographicalRangeIndexes rangeIndexes = indexSupplier.as(LexicographicalRangeIndexes.class); + if (rangeIndexes != null) { + final BitmapColumnIndex rangeBitmaps = rangeIndexes.forRange( boundDimFilter.getLower(), boundDimFilter.isLowerStrict(), boundDimFilter.getUpper(), boundDimFilter.isUpperStrict() ); - if (rangeBitmaps != null) { - // preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set - if (doesMatchNull()) { - return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps); - } else { - return rangeBitmaps; - } + if (rangeBitmaps == null) { + return null; + } + // preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set + if (doesMatchNull()) { + return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps); + } else { + return rangeBitmaps; } } } if (supportNumericShortCircuit()) { - final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(boundDimFilter.getDimension()); - if (indexSupplier == null) { - return Filters.makeNullIndex(doesMatchNull(), selector); - } - final NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); - if (rangeIndex != null) { + final NumericRangeIndexes rangeIndexes = indexSupplier.as(NumericRangeIndexes.class); + if (rangeIndexes != null) { final Number lower = boundDimFilter.hasLowerBound() ? Doubles.tryParse(boundDimFilter.getLower()) : null; final Number upper = boundDimFilter.hasUpperBound() ? Doubles.tryParse(boundDimFilter.getUpper()) : null; // valid number bounds are required to use the range index, otherwise we need to fall back to the predicate @@ -115,26 +119,31 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) final boolean lowerValid = !(boundDimFilter.hasLowerBound() && lower == null); final boolean upperValid = !(boundDimFilter.hasUpperBound() && upper == null); if (lowerValid && upperValid) { - final BitmapColumnIndex rangeBitmaps = rangeIndex.forRange( + final BitmapColumnIndex rangeBitmaps = rangeIndexes.forRange( lower, boundDimFilter.isLowerStrict(), upper, boundDimFilter.isUpperStrict() ); - if (rangeBitmaps != null) { - // preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set - if (doesMatchNull()) { - return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps); - } else { - return rangeBitmaps; - } + if (rangeBitmaps == null) { + return null; + } + // preserve sad backwards compatible behavior where bound filter matches 'null' if the lower bound is not set + if (doesMatchNull()) { + return wrapRangeIndexWithNullValueIndex(indexSupplier, rangeBitmaps); + } else { + return rangeBitmaps; } } } } - // fall back to predicate based index if it is available - return Filters.makePredicateIndex(boundDimFilter.getDimension(), selector, getPredicateFactory()); + final DruidPredicateIndexes predicateIndexes = indexSupplier.as(DruidPredicateIndexes.class); + if (predicateIndexes != null) { + return predicateIndexes.forPredicate(getPredicateFactory()); + } + // index doesn't exist + return null; } @Nullable @@ -150,7 +159,7 @@ private BitmapColumnIndex wrapRangeIndexWithNullValueIndex( if (nulls == null) { return null; } - nullBitmap = nulls.forNull(); + nullBitmap = nulls.get(); return new BitmapColumnIndex() { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java index b306f6bfa3cc..9be846786527 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.filter; import com.google.common.base.Preconditions; +import org.apache.druid.math.expr.Evals; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.Filter; @@ -34,9 +35,10 @@ import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionSelector; -import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.index.BitmapColumnIndex; import javax.annotation.Nullable; import java.util.ArrayList; @@ -233,6 +235,26 @@ public Supplier makeLongProcessor(BaseLongColumnValueSelector selector }; } + @Override + public Supplier makeArrayProcessor( + BaseObjectColumnValueSelector selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + return () -> { + final Object o = selector.getObject(); + if (o instanceof Object[]) { + final Object[] arr = (Object[]) o; + final String[] s = new String[arr.length]; + for (int i = 0; i < arr.length; i++) { + s[i] = Evals.asString(arr[i]); + } + return s; + } + return NULL_VALUE; + }; + } + @Override public Supplier makeComplexProcessor(BaseObjectColumnValueSelector selector) { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java index c11ae08eb3de..e5e6e9563a7f 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java @@ -39,7 +39,7 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; 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 a8f16e47f95c..711395cfe820 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 @@ -46,10 +46,12 @@ import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.virtual.ExpressionSelectors; import org.apache.druid.segment.virtual.ExpressionVectorSelectors; @@ -122,6 +124,11 @@ public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) ColumnCapabilitiesImpl.createSimpleSingleValueStringColumnCapabilities(), ExpressionVectorSelectors.makeVectorObjectSelector(factory, theExpr) ).makeMatcher(predicateFactory); + case ARRAY: + return VectorValueMatcherColumnProcessorFactory.instance().makeObjectProcessor( + ColumnCapabilitiesImpl.createDefault().setType(ExpressionType.toColumnType(outputType)).setHasNulls(true), + ExpressionVectorSelectors.makeVectorObjectSelector(factory, theExpr) + ).makeMatcher(predicateFactory); default: if (ExpressionType.NESTED_DATA.equals(outputType)) { return VectorValueMatcherColumnProcessorFactory.instance().makeObjectProcessor( @@ -187,7 +194,7 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) final Expr.BindingAnalysis details = bindingDetails.get(); if (details.getRequiredBindings().isEmpty()) { // Constant expression. - return Filters.makeNullIndex( + return Filters.makeMissingColumnNullIndex( expr.get().eval(InputBindings.nilBindings()).asBoolean(), selector ); @@ -199,10 +206,7 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) // we use a default 'all false' capabilities here because if the column has a bitmap index, but the capabilities // are null, it means that the column is missing and should take the single valued path, while truly unknown // things will not have a bitmap index available - final ColumnCapabilities capabilities = selector.getColumnCapabilitiesWithDefault( - column, - ColumnCapabilitiesImpl.createDefault() - ); + final ColumnCapabilities capabilities = selector.getColumnCapabilities(column); if (ExpressionSelectors.canMapOverDictionary(details, capabilities)) { if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) { return null; @@ -210,7 +214,7 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) return Filters.makePredicateIndex( column, selector, - getBitmapPredicateFactory() + getBitmapPredicateFactory(capabilities) ); } } @@ -338,7 +342,7 @@ public boolean equals(Object obj) * {@link DruidPredicateFactory} which evaluates the expression using the value as input, used for building predicate * indexes where the raw column values will be checked against this predicate */ - private DruidPredicateFactory getBitmapPredicateFactory() + private DruidPredicateFactory getBitmapPredicateFactory(@Nullable ColumnCapabilities inputCapabilites) { return new DruidPredicateFactory() { @@ -414,6 +418,19 @@ public boolean applyNull() }; } + @Override + public Predicate makeArrayPredicate(@Nullable TypeSignature arrayType) + { + if (inputCapabilites == null) { + return input -> expr.get() + .eval(InputBindings.forInputSupplier(ExpressionType.STRING_ARRAY, () -> input)) + .asBoolean(); + } + return input -> expr.get().eval( + InputBindings.forInputSupplier(ExpressionType.fromColumnType(inputCapabilites), () -> input) + ).asBoolean(); + } + // The hashcode and equals are to make SubclassesMustOverrideEqualsAndHashCodeTest stop complaining.. // DruidPredicateFactory currently doesn't really need equals or hashcode since 'toString' method that is actually // called when testing equality of DimensionPredicateFilter, so it's the truly required method, but that seems diff --git a/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java index 295f7c744263..cbc2b21b2875 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java @@ -27,8 +27,8 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.AllFalseBitmapColumnIndex; -import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; 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 7f9abfae2fa0..c522f8b4aa7a 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 @@ -35,16 +35,16 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.AllFalseBitmapColumnIndex; -import org.apache.druid.segment.column.AllTrueBitmapColumnIndex; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; import org.apache.druid.segment.filter.cnf.CalciteCnfHelper; import org.apache.druid.segment.filter.cnf.HiveCnfHelper; +import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; import org.apache.druid.segment.join.filter.AllNullColumnSelectorFactory; import javax.annotation.Nullable; @@ -90,29 +90,6 @@ public static Filter toFilter(@Nullable DimFilter dimFilter) return dimFilter == null ? null : dimFilter.toOptimizedFilter(); } - /** - * Create a ValueMatcher that compares row values to the provided string. - *

- * An implementation of this method should be able to handle dimensions of various types. - * - * @param columnSelectorFactory Selector for columns. - * @param columnName The column to filter. - * @param value The value to match against, represented as a String. - * - * @return An object that matches row values on the provided value. - */ - public static ValueMatcher makeValueMatcher( - final ColumnSelectorFactory columnSelectorFactory, - final String columnName, - final String value - ) - { - return ColumnProcessors.makeProcessor( - columnName, - new ConstantValueMatcherFactory(value), - columnSelectorFactory - ); - } /** * Create a ValueMatcher that applies a predicate to row values. @@ -154,9 +131,9 @@ public static BitmapColumnIndex makePredicateIndex( Preconditions.checkNotNull(predicateFactory, "predicateFactory"); final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column); if (indexSupplier != null) { - final DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); - if (predicateIndex != null) { - return predicateIndex.forPredicate(predicateFactory); + final DruidPredicateIndexes predicateIndexes = indexSupplier.as(DruidPredicateIndexes.class); + if (predicateIndexes != null) { + return predicateIndexes.forPredicate(predicateFactory); } // index doesn't exist return null; @@ -167,7 +144,7 @@ public static BitmapColumnIndex makePredicateIndex( : new AllFalseBitmapColumnIndex(selector); } - public static BitmapColumnIndex makeNullIndex(boolean matchesNull, final ColumnIndexSelector selector) + public static BitmapColumnIndex makeMissingColumnNullIndex(boolean matchesNull, final ColumnIndexSelector selector) { return matchesNull ? new AllTrueBitmapColumnIndex(selector) : new AllFalseBitmapColumnIndex(selector); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java index 5ae1679d0458..05e357970cde 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java @@ -27,7 +27,7 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; import javax.annotation.Nullable; import java.util.Objects; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java index e4ffb7f428f5..ff3f9b51e96e 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java @@ -34,12 +34,12 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.AllFalseBitmapColumnIndex; -import org.apache.druid.segment.column.AllTrueBitmapColumnIndex; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -82,23 +82,23 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) : new AllFalseBitmapColumnIndex(selector); } if (isSimpleEquals()) { - StringValueSetIndex valueIndex = indexSupplier.as(StringValueSetIndex.class); - if (valueIndex != null) { - return valueIndex.forValue( + StringValueSetIndexes valueIndexes = indexSupplier.as(StringValueSetIndexes.class); + if (valueIndexes != null) { + return valueIndexes.forValue( NullHandling.emptyToNullIfNeeded(likeMatcher.getPrefix()) ); } } if (isSimplePrefix()) { - final LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); - if (rangeIndex != null) { + final LexicographicalRangeIndexes rangeIndexes = indexSupplier.as(LexicographicalRangeIndexes.class); + if (rangeIndexes != null) { final String lower = NullHandling.nullToEmptyIfNeeded(likeMatcher.getPrefix()); final String upper = NullHandling.nullToEmptyIfNeeded(likeMatcher.getPrefix()) + Character.MAX_VALUE; if (likeMatcher.getSuffixMatch() == LikeDimFilter.LikeMatcher.SuffixMatch.MATCH_ANY) { - return rangeIndex.forRange(lower, false, upper, false); + return rangeIndexes.forRange(lower, false, upper, false); } else { - return rangeIndex.forRange(lower, false, upper, false, likeMatcher::matchesSuffixOnly); + return rangeIndexes.forRange(lower, false, upper, false, likeMatcher::matchesSuffixOnly); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java index b508dd45a1ad..2201c06c4410 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java @@ -32,8 +32,8 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexCapabilities; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java index 77110f741a6e..f5840d3d0c8c 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java @@ -39,9 +39,9 @@ import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java index 6ccd46648f7f..36b97d6d4d5b 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java @@ -22,6 +22,7 @@ import com.google.common.base.Predicate; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Rows; +import org.apache.druid.math.expr.ExprEval; import org.apache.druid.query.filter.DruidDoublePredicate; import org.apache.druid.query.filter.DruidFloatPredicate; import org.apache.druid.query.filter.DruidLongPredicate; @@ -35,8 +36,10 @@ import org.apache.druid.segment.ColumnProcessorFactory; import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.NilColumnValueSelector; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; +import javax.annotation.Nullable; import java.util.List; /** @@ -46,7 +49,7 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + if (selector instanceof NilColumnValueSelector) { + // Column does not exist, or is unfilterable. Treat it as all nulls. + return BooleanValueMatcher.of(predicateFactory.makeArrayPredicate(columnCapabilities).apply(null)); + } else { + // use the object predicate + final Predicate predicate = predicateFactory.makeArrayPredicate(columnCapabilities); + return new ValueMatcher() + { + @Override + public boolean matches() + { + Object o = selector.getObject(); + if (o == null || o instanceof Object[]) { + return predicate.apply((Object[]) o); + } + if (o instanceof List) { + ExprEval oEval = ExprEval.bestEffortArray((List) o); + return predicate.apply(oEval.asArray()); + } + // upcast non-array to a single element array to behave consistently with expressions.. idk if this is cool + return predicate.apply(new Object[]{o}); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("predicate", predicate); + } + }; + } + } + @Override public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector selector) { @@ -117,6 +159,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) private DruidLongPredicate longPredicate; private DruidFloatPredicate floatPredicate; private DruidDoublePredicate doublePredicate; + private Predicate arrayPredicate; @Override public boolean matches() @@ -134,6 +177,8 @@ public boolean matches() } else if (rowValue instanceof Number) { // Double or some other non-int, non-long, non-float number. return getDoublePredicate().applyDouble((double) rowValue); + } else if (rowValue instanceof Object[]) { + return getArrayPredicate().apply((Object[]) rowValue); } else { // Other types. Cast to list of strings and evaluate them as strings. // Boolean values are handled here as well since it is not a known type in Druid. @@ -196,6 +241,14 @@ private DruidDoublePredicate getDoublePredicate() return doublePredicate; } + + private Predicate getArrayPredicate() + { + if (arrayPredicate == null) { + arrayPredicate = predicateFactory.makeArrayPredicate(null); + } + return arrayPredicate; + } }; } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java index 036059d95c75..a5307befb365 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/SelectorFilter.java @@ -33,10 +33,10 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; @@ -87,28 +87,32 @@ public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) final boolean isNull = NullHandling.isNullOrEquivalent(value); final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(dimension); if (indexSupplier == null) { - return Filters.makeNullIndex(isNull, selector); + return Filters.makeMissingColumnNullIndex(isNull, selector); } if (isNull) { final NullValueIndex nullValueIndex = indexSupplier.as(NullValueIndex.class); if (nullValueIndex == null) { return null; } - return nullValueIndex.forNull(); + return nullValueIndex.get(); } else { - final StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); - if (valueSetIndex == null) { + final StringValueSetIndexes valueSetIndexes = indexSupplier.as(StringValueSetIndexes.class); + if (valueSetIndexes == null) { // column exists, but has no index return null; } - return valueSetIndex.forValue(value); + return valueSetIndexes.forValue(value); } } @Override public ValueMatcher makeMatcher(ColumnSelectorFactory factory) { - return Filters.makeValueMatcher(factory, dimension, value); + return ColumnProcessors.makeProcessor( + dimension, + new StringConstantValueMatcherFactory(value), + factory + ); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java index 1efa22f1f0ae..b174b7f4ddcc 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java @@ -36,13 +36,13 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.AllFalseBitmapColumnIndex; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; -import org.apache.druid.segment.column.SpatialIndex; import org.apache.druid.segment.incremental.SpatialDimensionRowTransformer; +import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.SpatialIndex; import javax.annotation.Nullable; import java.util.Objects; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ConstantValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/segment/filter/StringConstantValueMatcherFactory.java similarity index 80% rename from processing/src/main/java/org/apache/druid/segment/filter/ConstantValueMatcherFactory.java rename to processing/src/main/java/org/apache/druid/segment/filter/StringConstantValueMatcherFactory.java index 7dae76cf5422..a1a722e25273 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ConstantValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/StringConstantValueMatcherFactory.java @@ -28,6 +28,7 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnProcessorFactory; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import javax.annotation.Nullable; @@ -35,12 +36,12 @@ /** * Creates {@link ValueMatcher} that match constants. */ -public class ConstantValueMatcherFactory implements ColumnProcessorFactory +public class StringConstantValueMatcherFactory implements ColumnProcessorFactory { @Nullable private final String matchValue; - ConstantValueMatcherFactory(@Nullable String matchValue) + StringConstantValueMatcherFactory(@Nullable String matchValue) { this.matchValue = NullHandling.emptyToNullIfNeeded(matchValue); } @@ -76,6 +77,18 @@ public ValueMatcher makeLongProcessor(BaseLongColumnValueSelector selector) return ValueMatchers.makeLongValueMatcher(selector, matchValue); } + @Override + public ValueMatcher makeArrayProcessor( + BaseObjectColumnValueSelector selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + // this is gonna fail because SelectorPredicateFactory does not implement array predicate... + return new PredicateValueMatcherFactory( + new SelectorPredicateFactory(matchValue) + ).makeArrayProcessor(selector, columnCapabilities); + } + @Override public ValueMatcher makeComplexProcessor(BaseObjectColumnValueSelector selector) { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java index 079d2f5d103a..b363b384331c 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java @@ -27,8 +27,8 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; -import org.apache.druid.segment.column.AllTrueBitmapColumnIndex; -import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java b/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java index 2d09680a4510..75b4dcc77f45 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java @@ -39,7 +39,7 @@ import java.util.Objects; /** - * Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link ConstantValueMatcherFactory} + * Utility methods for creating {@link ValueMatcher} instances. Mainly used by {@link StringConstantValueMatcherFactory} * and {@link PredicateValueMatcherFactory}. */ public class ValueMatchers @@ -114,8 +114,22 @@ public static ValueMatcher makeFloatValueMatcher( return makeNumericNullValueMatcher(selector); } + return makeFloatValueMatcher(selector, matchVal); + } + + /** + * Creates a constant-based {@link ValueMatcher} for a float-typed selector. + * + * @param selector column selector + * @param value value to match + */ + public static ValueMatcher makeFloatValueMatcher( + final BaseFloatColumnValueSelector selector, + final float value + ) + { // Use "floatToIntBits" to canonicalize NaN values. - final int matchValIntBits = Float.floatToIntBits(matchVal); + final int matchValIntBits = Float.floatToIntBits(value); return new ValueMatcher() { @Override @@ -141,7 +155,11 @@ public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelecto if (matchVal == null) { return makeNumericNullValueMatcher(selector); } - final long matchValLong = matchVal; + return makeLongValueMatcher(selector, matchVal); + } + + public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, long value) + { return new ValueMatcher() { @Override @@ -150,7 +168,7 @@ public boolean matches() if (selector.isNull()) { return false; } - return selector.getLong() == matchValLong; + return selector.getLong() == value; } @Override @@ -187,6 +205,7 @@ public void inspectRuntimeShape(RuntimeShapeInspector inspector) }; } + /** * Creates a predicate-based {@link ValueMatcher} for a float-typed selector. * @@ -235,8 +254,16 @@ public static ValueMatcher makeDoubleValueMatcher( return makeNumericNullValueMatcher(selector); } + return makeDoubleValueMatcher(selector, matchVal); + } + + public static ValueMatcher makeDoubleValueMatcher( + final BaseDoubleColumnValueSelector selector, + final double value + ) + { // Use "doubleToLongBits" to canonicalize NaN values. - final long matchValLongBits = Double.doubleToLongBits(matchVal); + final long matchValLongBits = Double.doubleToLongBits(value); return new ValueMatcher() { @Override diff --git a/processing/src/main/java/org/apache/druid/segment/column/AllFalseBitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/AllFalseBitmapColumnIndex.java similarity index 90% rename from processing/src/main/java/org/apache/druid/segment/column/AllFalseBitmapColumnIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/AllFalseBitmapColumnIndex.java index 4c6b0f301cc2..fb986c51d97b 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/AllFalseBitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/AllFalseBitmapColumnIndex.java @@ -17,10 +17,12 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.filter.ColumnIndexSelector; +import org.apache.druid.segment.column.ColumnIndexCapabilities; +import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; public class AllFalseBitmapColumnIndex implements BitmapColumnIndex { diff --git a/processing/src/main/java/org/apache/druid/segment/column/AllTrueBitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java similarity index 90% rename from processing/src/main/java/org/apache/druid/segment/column/AllTrueBitmapColumnIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java index d129f814afa7..686add0693ce 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/AllTrueBitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java @@ -17,10 +17,12 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.filter.ColumnIndexSelector; +import org.apache.druid.segment.column.ColumnIndexCapabilities; +import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; public class AllTrueBitmapColumnIndex implements BitmapColumnIndex { diff --git a/processing/src/main/java/org/apache/druid/segment/column/BitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java similarity index 91% rename from processing/src/main/java/org/apache/druid/segment/column/BitmapColumnIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java index 255e19de4cac..77b78ac0698e 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/BitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java @@ -17,9 +17,10 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.segment.column.ColumnIndexCapabilities; public interface BitmapColumnIndex { diff --git a/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDictionaryEncodedStringValueIndex.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDictionaryEncodedStringValueIndex.java similarity index 94% rename from processing/src/main/java/org/apache/druid/segment/column/IndexedStringDictionaryEncodedStringValueIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/IndexedStringDictionaryEncodedStringValueIndex.java index 668694bb4a13..c3c0c304410c 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDictionaryEncodedStringValueIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDictionaryEncodedStringValueIndex.java @@ -17,11 +17,12 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import javax.annotation.Nullable; diff --git a/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDruidPredicateIndex.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java similarity index 89% rename from processing/src/main/java/org/apache/druid/segment/column/IndexedStringDruidPredicateIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java index 9c5aa9af1212..9038ed560e11 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/IndexedStringDruidPredicateIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java @@ -17,19 +17,23 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import com.google.common.base.Predicate; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; import javax.annotation.Nullable; import java.util.Iterator; import java.util.NoSuchElementException; -public final class IndexedStringDruidPredicateIndex> implements DruidPredicateIndex +public final class IndexedStringDruidPredicateIndexes> implements + DruidPredicateIndexes { private final BitmapFactory bitmapFactory; private final TDictionary dictionary; @@ -37,7 +41,7 @@ public final class IndexedStringDruidPredicateIndex bitmaps, diff --git a/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8LexicographicalRangeIndex.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java similarity index 94% rename from processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8LexicographicalRangeIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java index 5ddb01ba02c6..0dde5754bbe4 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8LexicographicalRangeIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; @@ -29,15 +29,18 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.IntListUtils; +import org.apache.druid.segment.column.ColumnConfig; +import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.util.Iterator; import java.util.NoSuchElementException; -public final class IndexedUtf8LexicographicalRangeIndex> - implements LexicographicalRangeIndex +public final class IndexedUtf8LexicographicalRangeIndexes> + implements LexicographicalRangeIndexes { private final BitmapFactory bitmapFactory; private final TDictionary dictionary; @@ -47,7 +50,7 @@ public final class IndexedUtf8LexicographicalRangeIndex bitmaps, diff --git a/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8ValueSetIndex.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java similarity index 83% rename from processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8ValueSetIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java index c568e78d9b07..251059747013 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/IndexedUtf8ValueSetIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; @@ -27,8 +27,15 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.java.util.common.ByteBufferUtils; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; +import org.apache.druid.segment.index.semantic.Utf8ValueSetIndexes; +import org.apache.druid.segment.index.semantic.ValueIndexes; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -37,8 +44,8 @@ import java.util.NoSuchElementException; import java.util.SortedSet; -public final class IndexedUtf8ValueSetIndex> - implements StringValueSetIndex, Utf8ValueSetIndex +public final class IndexedUtf8ValueIndexes> + implements StringValueSetIndexes, Utf8ValueSetIndexes, ValueIndexes { // This determines the cut-off point to switch the merging algorithm from doing binary-search per element in the value // set to doing a sorted merge algorithm between value set and dictionary. The ratio here represents the ratio b/w @@ -53,7 +60,7 @@ public final class IndexedUtf8ValueSetIndex bitmaps; - public IndexedUtf8ValueSetIndex( + public IndexedUtf8ValueIndexes( BitmapFactory bitmapFactory, TDictionary dictionary, Indexed bitmaps @@ -68,6 +75,7 @@ public IndexedUtf8ValueSetIndex( @Override public BitmapColumnIndex forValue(@Nullable String value) { + final ByteBuffer utf8 = StringUtils.toUtf8ByteBuffer(value); return new SimpleBitmapColumnIndex() { @Override @@ -85,20 +93,33 @@ public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) private ImmutableBitmap getBitmapForValue() { - final ByteBuffer valueUtf8 = value == null ? null : ByteBuffer.wrap(StringUtils.toUtf8(value)); - final int idx = dictionary.indexOf(valueUtf8); + final int idx = dictionary.indexOf(utf8); return getBitmap(idx); } }; } + @Nullable + @Override + public BitmapColumnIndex forValue(Object value, TypeSignature valueType) + { + if (valueType.isPrimitive()) { + return forValue( + ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value) + .castTo(ExpressionType.STRING) + .asString() + ); + } + return null; + } + @Override public BitmapColumnIndex forSortedValues(SortedSet values) { return getBitmapColumnIndexForSortedIterableUtf8( Iterables.transform( values, - input -> input != null ? ByteBuffer.wrap(StringUtils.toUtf8(input)) : null + StringUtils::toUtf8ByteBuffer ), values.size() ); @@ -196,6 +217,15 @@ private void findNext() // if the size of in-filter values is less than the threshold percentage of dictionary size, then use binary search // based lookup per value. The algorithm works well for smaller number of values. + return getSimpleImmutableBitmapIterableIndexFromIterator(valuesUtf8); + } + + /** + * Iterates over the value set, using binary search to look up each element. The algorithm works well for smaller + * number of values, and must be used if the values are not sorted in the same manner as {@link #dictionary} + */ + private SimpleImmutableBitmapIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable valuesUtf8) + { return new SimpleImmutableBitmapIterableIndex() { @Override diff --git a/processing/src/main/java/org/apache/druid/segment/column/SimpleBitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/SimpleBitmapColumnIndex.java similarity index 87% rename from processing/src/main/java/org/apache/druid/segment/column/SimpleBitmapColumnIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/SimpleBitmapColumnIndex.java index 9f494d0cd21a..29f04007a74a 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SimpleBitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/SimpleBitmapColumnIndex.java @@ -17,7 +17,10 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; + +import org.apache.druid.segment.column.ColumnIndexCapabilities; +import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; /** * {@link BitmapColumnIndex} with Druids "default" {@link ColumnIndexCapabilities}. diff --git a/processing/src/main/java/org/apache/druid/segment/column/SimpleImmutableBitmapIndex.java b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIndex.java similarity index 97% rename from processing/src/main/java/org/apache/druid/segment/column/SimpleImmutableBitmapIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIndex.java index 411e79e5761c..c314d3c329af 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SimpleImmutableBitmapIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIndex.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.BitmapResultFactory; diff --git a/processing/src/main/java/org/apache/druid/segment/column/SimpleImmutableBitmapIterableIndex.java b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java similarity index 97% rename from processing/src/main/java/org/apache/druid/segment/column/SimpleImmutableBitmapIterableIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java index 67587725f763..2a6d47c1b537 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SimpleImmutableBitmapIterableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.BitmapResultFactory; diff --git a/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedStringValueIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedStringValueIndex.java similarity index 85% rename from processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedStringValueIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedStringValueIndex.java index 6913ac998948..f7b76289a604 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedStringValueIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedStringValueIndex.java @@ -17,9 +17,10 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.segment.column.DictionaryEncodedColumn; import javax.annotation.Nullable; @@ -28,8 +29,8 @@ * to directly retrieve bitmaps via dictionary ids, as well as access to lower level details of such a column like * value lookup and value cardinality. * - * Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndex}, - * {@link LexicographicalRangeIndex}, {@link NumericRangeIndex}, or {@link DruidPredicateIndex} + * Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndexes}, + * {@link LexicographicalRangeIndexes}, {@link NumericRangeIndexes}, or {@link DruidPredicateIndexes} */ public interface DictionaryEncodedStringValueIndex extends DictionaryEncodedValueIndex { diff --git a/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedValueIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedValueIndex.java similarity index 85% rename from processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedValueIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedValueIndex.java index b1f0115062fd..b9e997e9d43a 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/DictionaryEncodedValueIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedValueIndex.java @@ -17,9 +17,10 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.segment.column.DictionaryEncodedColumn; /** * This exposes a 'raw' view into bitmap value indexes for {@link DictionaryEncodedColumn}. This allows callers @@ -29,8 +30,8 @@ * either already know what value the dictionary id represents, not care at all, or have some other means to know * exactly which bitmaps to retrieve. * - * Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndex}, - * {@link LexicographicalRangeIndex}, {@link NumericRangeIndex}, or {@link DruidPredicateIndex}. + * Most filter implementations should likely be using higher level index instead, such as {@link StringValueSetIndexes}, + * {@link LexicographicalRangeIndexes}, {@link NumericRangeIndexes}, or {@link DruidPredicateIndexes}. */ public interface DictionaryEncodedValueIndex { diff --git a/processing/src/main/java/org/apache/druid/segment/column/DruidPredicateIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/DruidPredicateIndexes.java similarity index 90% rename from processing/src/main/java/org/apache/druid/segment/column/DruidPredicateIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/DruidPredicateIndexes.java index 44f69279aa9e..c7e87ec20f1e 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/DruidPredicateIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/DruidPredicateIndexes.java @@ -17,16 +17,17 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.segment.index.BitmapColumnIndex; import javax.annotation.Nullable; /** * Uses a {@link DruidPredicateFactory} to construct a {@link BitmapColumnIndex} */ -public interface DruidPredicateIndex +public interface DruidPredicateIndexes { /** * Get a {@link BitmapColumnIndex} corresponding to all the rows that match the supplied {@link DruidPredicateFactory} diff --git a/processing/src/main/java/org/apache/druid/segment/column/LexicographicalRangeIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/LexicographicalRangeIndexes.java similarity index 95% rename from processing/src/main/java/org/apache/druid/segment/column/LexicographicalRangeIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/LexicographicalRangeIndexes.java index 83c8fcfabd54..2d3b71c49d84 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/LexicographicalRangeIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/LexicographicalRangeIndexes.java @@ -17,9 +17,10 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; import com.google.common.base.Predicate; +import org.apache.druid.segment.index.BitmapColumnIndex; import javax.annotation.Nullable; @@ -28,7 +29,7 @@ * allowing short-circuit processing of string value ranges. This index does not match null values, union the results * of this index with {@link NullValueIndex} if null values should be considered part of the value range. */ -public interface LexicographicalRangeIndex +public interface LexicographicalRangeIndexes { /** * Get a {@link BitmapColumnIndex} corresponding to the values supplied in the specified range. If supplied starting diff --git a/processing/src/main/java/org/apache/druid/segment/column/NullValueIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/NullValueIndex.java similarity index 87% rename from processing/src/main/java/org/apache/druid/segment/column/NullValueIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/NullValueIndex.java index 65ab439b56df..8768caa54627 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/NullValueIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/NullValueIndex.java @@ -17,12 +17,14 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; + +import org.apache.druid.segment.index.BitmapColumnIndex; /** * Provides index for all null rows in a column, to use with IS/IS NOT NULL filters */ public interface NullValueIndex { - BitmapColumnIndex forNull(); + BitmapColumnIndex get(); } diff --git a/processing/src/main/java/org/apache/druid/segment/column/NumericRangeIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/NumericRangeIndexes.java similarity index 92% rename from processing/src/main/java/org/apache/druid/segment/column/NumericRangeIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/NumericRangeIndexes.java index cebece48b2cc..7f3caa75d2a7 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/NumericRangeIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/NumericRangeIndexes.java @@ -17,7 +17,9 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; + +import org.apache.druid.segment.index.BitmapColumnIndex; import javax.annotation.Nullable; @@ -26,7 +28,7 @@ * This index does not match null values, union the results of this index with {@link NullValueIndex} if null values * should be considered part of the value range. */ -public interface NumericRangeIndex +public interface NumericRangeIndexes { /** * Get a {@link BitmapColumnIndex} corresponding to the values supplied in the specified range. If supplied starting diff --git a/processing/src/main/java/org/apache/druid/segment/column/SpatialIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/SpatialIndex.java similarity index 95% rename from processing/src/main/java/org/apache/druid/segment/column/SpatialIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/SpatialIndex.java index 462a15a079b3..cf19d5c4d9a0 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/SpatialIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/SpatialIndex.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; import org.apache.druid.collections.spatial.ImmutableRTree; diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringValueSetIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/StringValueSetIndexes.java similarity index 91% rename from processing/src/main/java/org/apache/druid/segment/column/StringValueSetIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/StringValueSetIndexes.java index 8ceca2d86add..acb4b6712716 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/StringValueSetIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/StringValueSetIndexes.java @@ -17,9 +17,10 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.segment.index.BitmapColumnIndex; import javax.annotation.Nullable; import java.util.SortedSet; @@ -27,7 +28,7 @@ /** * Index on individual values, and provides bitmaps for the rows which contain these values */ -public interface StringValueSetIndex +public interface StringValueSetIndexes { /** * Get the {@link ImmutableBitmap} corresponding to the supplied value. Generates an empty bitmap when passed a diff --git a/processing/src/main/java/org/apache/druid/segment/column/Utf8ValueSetIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/Utf8ValueSetIndexes.java similarity index 89% rename from processing/src/main/java/org/apache/druid/segment/column/Utf8ValueSetIndex.java rename to processing/src/main/java/org/apache/druid/segment/index/semantic/Utf8ValueSetIndexes.java index 6598e36f2069..1bf2792e444d 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/Utf8ValueSetIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/Utf8ValueSetIndexes.java @@ -17,14 +17,15 @@ * under the License. */ -package org.apache.druid.segment.column; +package org.apache.druid.segment.index.semantic; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.segment.index.BitmapColumnIndex; import java.nio.ByteBuffer; import java.util.SortedSet; -public interface Utf8ValueSetIndex +public interface Utf8ValueSetIndexes { /** * Get an {@link Iterable} of {@link ImmutableBitmap} corresponding to the specified set of values (if they are diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/ValueIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/ValueIndexes.java new file mode 100644 index 000000000000..28fcf0ae9b17 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/ValueIndexes.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.index.semantic; + +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.index.BitmapColumnIndex; + +import javax.annotation.Nullable; + +public interface ValueIndexes +{ + + /** + * Get the {@link ImmutableBitmap} corresponding to the supplied value. Generates an empty bitmap when passed a + * value that doesn't exist. May return null if a value index cannot be computed for the supplied value type. + * + * @param value value to match + * @param valueType type of the value to match, used to assist conversion from the match value type to the column + * value type + * @return {@link ImmutableBitmap} corresponding to the rows which match the value, or null if an index + * connot be computed for the supplied value type + */ + @Nullable + BitmapColumnIndex forValue(Object value, TypeSignature valueType); +} 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 c1703640330f..319d4a81ef1b 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 @@ -37,6 +37,7 @@ import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.join.Equality; @@ -112,6 +113,15 @@ public Supplier makeLongProcessor(BaseLongColumnValueSelector selector) } } + @Override + public Supplier makeArrayProcessor( + BaseObjectColumnValueSelector selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + throw new QueryUnsupportedException("Joining against a ARRAY columns is not supported."); + } + @Override public Supplier makeComplexProcessor(BaseObjectColumnValueSelector selector) { 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 5004c897a466..a433a0ae5522 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 @@ -48,6 +48,7 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.SimpleDescendingOffset; import org.apache.druid.segment.SimpleSettableOffset; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.IndexedInts; @@ -491,6 +492,17 @@ public ConditionMatcher makeLongProcessor(BaseLongColumnValueSelector selector) } } + @Override + public ConditionMatcher makeArrayProcessor( + BaseObjectColumnValueSelector selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + return () -> { + throw new QueryUnsupportedException("Joining against ARRAY columns is not supported."); + }; + } + @Override public ConditionMatcher makeComplexProcessor(BaseObjectColumnValueSelector selector) { diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java index 59c0070d2430..abb91fc1483f 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedCommonFormatColumn.java @@ -147,10 +147,9 @@ public ColumnCapabilities toColumnCapabilities() .setDictionaryValuesSorted(true) .setDictionaryValuesUnique(true) .setHasBitmapIndexes(true) - .setFilterable(true) .setHasNulls(hasNulls); } - return ColumnCapabilitiesImpl.createDefault().setType(logicalType).setHasNulls(hasNulls).setFilterable(true); + return ColumnCapabilitiesImpl.createDefault().setType(logicalType).setHasNulls(hasNulls); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java index f1f298e08897..a35aa93c25b7 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedDataComplexTypeSerde.java @@ -106,7 +106,6 @@ public void deserializeColumn( } builder.setComplexColumnSupplier(supplier); builder.setColumnFormat(new NestedColumnFormatV4()); - builder.setFilterable(true); } @Override @@ -188,7 +187,7 @@ public ColumnFormat merge(@Nullable ColumnFormat otherFormat) @Override public ColumnCapabilities toColumnCapabilities() { - return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA).setHasNulls(true).setFilterable(true); + return ColumnCapabilitiesImpl.createDefault().setType(ColumnType.NESTED_DATA).setHasNulls(true); } } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java index 465d8cb30c55..06d29bcf9e7e 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java @@ -44,23 +44,23 @@ import org.apache.druid.query.filter.DruidLongPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.IntListUtils; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.NumericRangeIndex; -import org.apache.druid.segment.column.SimpleBitmapColumnIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIterableIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.SimpleBitmapColumnIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.NumericRangeIndexes; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -132,7 +132,7 @@ public T as(Class clazz) if (clazz.equals(NullValueIndex.class)) { final BitmapColumnIndex nullIndex; if (localDictionarySupplier.get().get(0) == 0) { - // null index is always 0 in the global dictionary, even if there are no null rows in any of the literal columns + // null index is always 0 in the global dictionary, even if there are no null rows in any of the nested fields nullIndex = new SimpleImmutableBitmapIndex(bitmaps.get(0)); } else { nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap()); @@ -145,40 +145,40 @@ public T as(Class clazz) if (singleType != null) { switch (singleType.getType()) { case STRING: - if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedStringValueSetIndex(); - } else if (clazz.equals(LexicographicalRangeIndex.class)) { - return (T) new NestedStringLexicographicalRangeIndex(); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedStringPredicateIndex(); + if (clazz.equals(StringValueSetIndexes.class)) { + return (T) new NestedStringValueSetIndexes(); + } else if (clazz.equals(LexicographicalRangeIndexes.class)) { + return (T) new NestedStringLexicographicalRangeIndexes(); + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new NestedStringPredicateIndexes(); } return null; case LONG: - if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedLongValueSetIndex(); - } else if (clazz.equals(NumericRangeIndex.class)) { - return (T) new NestedLongNumericRangeIndex(); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedLongPredicateIndex(); + if (clazz.equals(StringValueSetIndexes.class)) { + return (T) new NestedLongStringValueSetIndex(); + } else if (clazz.equals(NumericRangeIndexes.class)) { + return (T) new NestedLongNumericRangeIndexes(); + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new NestedLongPredicateIndexes(); } return null; case DOUBLE: - if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedDoubleValueSetIndex(); - } else if (clazz.equals(NumericRangeIndex.class)) { - return (T) new NestedDoubleNumericRangeIndex(); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedDoublePredicateIndex(); + if (clazz.equals(StringValueSetIndexes.class)) { + return (T) new NestedDoubleStringValueSetIndex(); + } else if (clazz.equals(NumericRangeIndexes.class)) { + return (T) new NestedDoubleNumericRangeIndexes(); + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new NestedDoublePredicateIndexes(); } return null; default: return null; } } - if (clazz.equals(StringValueSetIndex.class)) { - return (T) new NestedVariantValueSetIndex(); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new NestedVariantPredicateIndex(); + if (clazz.equals(StringValueSetIndexes.class)) { + return (T) new NestedVariantStringValueSetIndexes(); + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new NestedVariantPredicateIndexes(); } return null; } @@ -353,7 +353,7 @@ public ImmutableBitmap getBitmap(int idx) } } - private class NestedStringValueSetIndex implements StringValueSetIndex + private class NestedStringValueSetIndexes implements StringValueSetIndexes { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -435,7 +435,7 @@ private void findNext() } } - private class NestedStringLexicographicalRangeIndex implements LexicographicalRangeIndex + private class NestedStringLexicographicalRangeIndexes implements LexicographicalRangeIndexes { @Override @Nullable @@ -533,7 +533,7 @@ public ImmutableBitmap next() } } - private class NestedStringPredicateIndex implements DruidPredicateIndex + private class NestedStringPredicateIndexes implements DruidPredicateIndexes { @Override @Nullable @@ -598,7 +598,7 @@ private void findNext() } } - private class NestedLongValueSetIndex implements StringValueSetIndex + private class NestedLongStringValueSetIndex implements StringValueSetIndexes { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -718,7 +718,7 @@ private void findNext() } } - private class NestedLongNumericRangeIndex implements NumericRangeIndex + private class NestedLongNumericRangeIndexes implements NumericRangeIndexes { @Override @Nullable @@ -741,7 +741,7 @@ public BitmapColumnIndex forRange( } } - private class NestedLongPredicateIndex implements DruidPredicateIndex + private class NestedLongPredicateIndexes implements DruidPredicateIndexes { @Override @Nullable @@ -811,7 +811,7 @@ private void findNext() } } - private class NestedDoubleValueSetIndex implements StringValueSetIndex + private class NestedDoubleStringValueSetIndex implements StringValueSetIndexes { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -930,7 +930,7 @@ private void findNext() } } - private class NestedDoubleNumericRangeIndex implements NumericRangeIndex + private class NestedDoubleNumericRangeIndexes implements NumericRangeIndexes { @Override @Nullable @@ -953,7 +953,7 @@ public BitmapColumnIndex forRange( } } - private class NestedDoublePredicateIndex implements DruidPredicateIndex + private class NestedDoublePredicateIndexes implements DruidPredicateIndexes { @Override @Nullable @@ -1023,7 +1023,7 @@ private void findNext() } } - private abstract class NestedVariantLiteralIndex + private abstract class NestedVariantIndexes { final FixedIndexed localDictionary = localDictionarySupplier.get(); final Indexed stringDictionary = globalStringDictionarySupplier.get(); @@ -1072,9 +1072,9 @@ IntList getIndexes(@Nullable String value) } /** - * {@link StringValueSetIndex} but for variant typed nested literal columns + * {@link StringValueSetIndexes} but for variant typed nested columns */ - private class NestedVariantValueSetIndex extends NestedVariantLiteralIndex implements StringValueSetIndex + private class NestedVariantStringValueSetIndexes extends NestedVariantIndexes implements StringValueSetIndexes { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -1151,9 +1151,9 @@ private void findNext() } /** - * {@link DruidPredicateIndex} but for variant typed nested literal columns + * {@link DruidPredicateIndexes} but for variant typed nested fields */ - private class NestedVariantPredicateIndex extends NestedVariantLiteralIndex implements DruidPredicateIndex + private class NestedVariantPredicateIndexes extends NestedVariantIndexes implements DruidPredicateIndexes { @Override @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java index b21760416fee..7dea845f707c 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java @@ -192,13 +192,21 @@ private int getIdFromGlobalDictionary(@Nullable String val) if (singleType != null) { switch (singleType.getType()) { case LONG: - final int globalLong = globalLongDictionary.indexOf(GuavaUtils.tryParseLong(val)); + final Long l = GuavaUtils.tryParseLong(val); + if (l == null) { + return -1; + } + final int globalLong = globalLongDictionary.indexOf(l); if (globalLong < 0) { return -1; } return globalLong + adjustLongId; case DOUBLE: - final int globalDouble = globalDoubleDictionary.indexOf(Doubles.tryParse(val)); + final Double d = Doubles.tryParse(val); + if (d == null) { + return -1; + } + final int globalDouble = globalDoubleDictionary.indexOf(d); if (globalDouble < 0) { return -1; } @@ -209,15 +217,21 @@ private int getIdFromGlobalDictionary(@Nullable String val) } else { int candidate = globalDictionary.indexOf(StringUtils.toUtf8ByteBuffer(val)); if (candidate < 0) { - candidate = globalLongDictionary.indexOf(GuavaUtils.tryParseLong(val)); - if (candidate >= 0) { - candidate += adjustLongId; + final Long l = GuavaUtils.tryParseLong(val); + if (l != null) { + candidate = globalLongDictionary.indexOf(l); + if (candidate >= 0) { + candidate += adjustLongId; + } } } if (candidate < 0) { - candidate = globalDoubleDictionary.indexOf(Doubles.tryParse(val)); - if (candidate >= 0) { - candidate += adjustDoubleId; + final Double d = Doubles.tryParse(val); + if (d != null) { + candidate = globalDoubleDictionary.indexOf(d); + if (candidate >= 0) { + candidate += adjustDoubleId; + } } } return candidate; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java index 23c21b876bcf..5624025a20c0 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java @@ -34,30 +34,35 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.filter.DruidDoublePredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.IntListUtils; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.NumericRangeIndex; -import org.apache.druid.segment.column.SimpleBitmapColumnIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIterableIndex; -import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.ColumnarDoubles; import org.apache.druid.segment.data.CompressedColumnarDoublesSuppliers; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.SimpleBitmapColumnIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.NumericRangeIndexes; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; +import org.apache.druid.segment.index.semantic.ValueIndexes; import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; import javax.annotation.Nullable; @@ -191,15 +196,19 @@ public T as(Class clazz) nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); } return (T) (NullValueIndex) () -> nullIndex; - } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) - || clazz.equals(DictionaryEncodedValueIndex.class)) { - return (T) new DoubleDictionaryEncodedValueSetIndex(); - } else if (clazz.equals(StringValueSetIndex.class)) { - return (T) new DoubleValueSetIndex(); - } else if (clazz.equals(NumericRangeIndex.class)) { + } else if (clazz.equals(ValueIndexes.class)) { + return (T) new DoubleValueIndexes(); + } else if (clazz.equals(StringValueSetIndexes.class)) { + return (T) new DoubleStringValueSetIndexes(); + } else if (clazz.equals(NumericRangeIndexes.class)) { return (T) new DoubleNumericRangeIndex(); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new DoublePredicateIndex(); + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new DoublePredicateIndexes(); + } else if ( + clazz.equals(DictionaryEncodedStringValueIndex.class) || + clazz.equals(DictionaryEncodedValueIndex.class) + ) { + return (T) new DoubleDictionaryEncodedValueSetIndex(); } return null; @@ -215,7 +224,45 @@ private ImmutableBitmap getBitmap(int idx) return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap; } - private class DoubleValueSetIndex implements StringValueSetIndex + private class DoubleValueIndexes implements ValueIndexes + { + @Nullable + @Override + public BitmapColumnIndex forValue(Object value, TypeSignature valueType) + { + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value) + .castTo(ExpressionType.DOUBLE); + if (eval.isNumericNull()) { + return null; + } + final double doubleValue = eval.asDouble(); + return new SimpleBitmapColumnIndex() + { + final FixedIndexed dictionary = doubleDictionarySupplier.get(); + @Override + public double estimateSelectivity(int totalRows) + { + final int id = dictionary.indexOf(doubleValue); + if (id < 0) { + return 0.0; + } + return (double) getBitmap(id).size() / totalRows; + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + { + final int id = dictionary.indexOf(doubleValue); + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + return bitmapResultFactory.wrapDimensionValue(getBitmap(id)); + } + }; + } + } + + private class DoubleStringValueSetIndexes implements StringValueSetIndexes { @Override public BitmapColumnIndex forValue(@Nullable String value) @@ -356,7 +403,7 @@ private void findNext() } } - private class DoubleNumericRangeIndex implements NumericRangeIndex + private class DoubleNumericRangeIndex implements NumericRangeIndexes { @Nullable @Override @@ -406,7 +453,7 @@ public ImmutableBitmap next() } } - private class DoublePredicateIndex implements DruidPredicateIndex + private class DoublePredicateIndexes implements DruidPredicateIndexes { @Nullable @Override diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java index 39d6057aefcc..a5170ca9a58f 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java @@ -33,30 +33,35 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.filter.DruidLongPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.IntListUtils; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.NumericRangeIndex; -import org.apache.druid.segment.column.SimpleBitmapColumnIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIterableIndex; -import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.ColumnarLongs; import org.apache.druid.segment.data.CompressedColumnarLongsSupplier; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.SimpleBitmapColumnIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.NumericRangeIndexes; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; +import org.apache.druid.segment.index.semantic.ValueIndexes; import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; import javax.annotation.Nullable; @@ -191,15 +196,19 @@ public T as(Class clazz) nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); } return (T) (NullValueIndex) () -> nullIndex; - } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) - || clazz.equals(DictionaryEncodedValueIndex.class)) { - return (T) new LongDictionaryEncodedValueSetIndex(); - } else if (clazz.equals(StringValueSetIndex.class)) { - return (T) new LongValueSetIndex(); - } else if (clazz.equals(NumericRangeIndex.class)) { + } else if (clazz.equals(ValueIndexes.class)) { + return (T) new LongValueIndexes(); + } else if (clazz.equals(StringValueSetIndexes.class)) { + return (T) new LongStringValueSetIndexes(); + } else if (clazz.equals(NumericRangeIndexes.class)) { return (T) new LongNumericRangeIndex(); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new LongPredicateIndex(); + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new LongPredicateIndexes(); + } else if ( + clazz.equals(DictionaryEncodedStringValueIndex.class) || + clazz.equals(DictionaryEncodedValueIndex.class) + ) { + return (T) new LongDictionaryEncodedValueSetIndex(); } return null; @@ -215,7 +224,45 @@ private ImmutableBitmap getBitmap(int idx) return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap; } - private class LongValueSetIndex implements StringValueSetIndex + private class LongValueIndexes implements ValueIndexes + { + @Nullable + @Override + public BitmapColumnIndex forValue(Object value, TypeSignature valueType) + { + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value) + .castTo(ExpressionType.LONG); + if (eval.isNumericNull()) { + return null; + } + final long longValue = eval.asLong(); + return new SimpleBitmapColumnIndex() + { + final FixedIndexed dictionary = longDictionarySupplier.get(); + @Override + public double estimateSelectivity(int totalRows) + { + final int id = dictionary.indexOf(longValue); + if (id < 0) { + return 0.0; + } + return (double) getBitmap(id).size() / totalRows; + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + { + final int id = dictionary.indexOf(longValue); + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + return bitmapResultFactory.wrapDimensionValue(getBitmap(id)); + } + }; + } + } + + private class LongStringValueSetIndexes implements StringValueSetIndexes { final FixedIndexed dictionary = longDictionarySupplier.get(); int defaultValueIndex = dictionary.indexOf(NullHandling.defaultLongValue()); @@ -354,7 +401,7 @@ private void findNext() } } - private class LongNumericRangeIndex implements NumericRangeIndex + private class LongNumericRangeIndex implements NumericRangeIndexes { @Nullable @Override @@ -404,7 +451,7 @@ public ImmutableBitmap next() } } - private class LongPredicateIndex implements DruidPredicateIndex + private class LongPredicateIndexes implements DruidPredicateIndexes { @Nullable @Override diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java index bf09e9fe1574..d5406ba8c45c 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java @@ -23,6 +23,8 @@ import com.google.common.base.Predicates; import com.google.common.primitives.Doubles; import com.google.common.primitives.Floats; +import it.unimi.dsi.fastutil.ints.IntArraySet; +import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.java.util.common.IAE; @@ -295,21 +297,59 @@ public int lookupId(String val) if (candidate >= 0) { return candidate; } - candidate = longDictionary.indexOf(GuavaUtils.tryParseLong(val)); - if (candidate >= 0) { - candidate += adjustLongId; - return candidate; + final Long l = GuavaUtils.tryParseLong(val); + if (l != null) { + candidate = longDictionary.indexOf(l); + if (candidate >= 0) { + candidate += adjustLongId; + return candidate; + } } - candidate = doubleDictionary.indexOf(Doubles.tryParse(val)); - if (candidate >= 0) { - candidate += adjustDoubleId; - return candidate; + final Double d = Doubles.tryParse(val); + if (d != null) { + candidate = doubleDictionary.indexOf(d); + if (candidate >= 0) { + candidate += adjustDoubleId; + return candidate; + } } // not in here, we can't really do anything cool here return -1; } + + public IntSet lookupIds(String val) + { + IntSet intList = new IntArraySet(3); + if (val == null) { + intList.add(0); + return intList; + } + int candidate = stringDictionary.indexOf(StringUtils.toUtf8ByteBuffer(val)); + if (candidate >= 0) { + intList.add(candidate); + } + Long l = GuavaUtils.tryParseLong(val); + if (l != null) { + candidate = longDictionary.indexOf(l); + if (candidate >= 0) { + candidate += adjustLongId; + intList.add(candidate); + } + } + Double d = Doubles.tryParse(val); + if (d != null) { + candidate = doubleDictionary.indexOf(d); + if (candidate >= 0) { + candidate += adjustDoubleId; + intList.add(candidate); + } + } + + return intList; + } + @Override public int getCardinality() { @@ -428,14 +468,14 @@ public int getRowValue(int offset) public ValueMatcher makeValueMatcher(final @Nullable String value) { if (extractionFn == null) { - final int valueId = lookupId(value); - if (valueId >= 0) { + final IntSet valueIds = VariantColumn.this.lookupIds(value); + if (valueIds.size() > 0) { return new ValueMatcher() { @Override public boolean matches() { - return getRowValue() == valueId; + return valueIds.contains(getRowValue()); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java index 69246edf6677..5fabfefef6c4 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java @@ -22,18 +22,21 @@ import com.google.common.base.Supplier; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; -import org.apache.druid.segment.column.BitmapColumnIndex; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; import org.apache.druid.segment.column.StringEncodingStrategy; +import org.apache.druid.segment.column.TypeSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.ColumnarInts; import org.apache.druid.segment.data.CompressedVSizeColumnarIntsSupplier; @@ -42,7 +45,13 @@ import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.data.FrontCodedIntArrayIndexed; import org.apache.druid.segment.data.GenericIndexed; +import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.VByte; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.SimpleBitmapColumnIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.ValueIndexes; import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; import javax.annotation.Nullable; @@ -143,11 +152,21 @@ public static VariantColumnAndIndexSupplier read( columnName, NestedCommonFormatColumnSerializer.BITMAP_INDEX_FILE_NAME ); - GenericIndexed valueIndexes = GenericIndexed.read( + final GenericIndexed valueIndexes = GenericIndexed.read( valueIndexBuffer, bitmapSerdeFactory.getObjectStrategy(), columnBuilder.getFileMapper() ); + final ByteBuffer elementIndexBuffer = NestedCommonFormatColumnPartSerde.loadInternalFile( + mapper, + columnName, + NestedCommonFormatColumnSerializer.ARRAY_ELEMENT_BITMAP_INDEX_FILE_NAME + ); + final GenericIndexed arrayElementIndexes = GenericIndexed.read( + elementIndexBuffer, + bitmapSerdeFactory.getObjectStrategy(), + columnBuilder.getFileMapper() + ); longDictionarySupplier = FixedIndexed.read( longDictionaryBuffer, @@ -184,6 +203,7 @@ public static VariantColumnAndIndexSupplier read( arrayDictionarySupplier, ints, valueIndexes, + arrayElementIndexes, bitmapSerdeFactory.getBitmapFactory(), columnConfig, size @@ -201,7 +221,7 @@ public static VariantColumnAndIndexSupplier read( private final ColumnType logicalType; @Nullable private final Byte variantTypeSetByte; - + private final BitmapFactory bitmapFactory; private final GenericIndexed stringDictionary; private final Supplier frontCodedStringDictionarySupplier; private final Supplier> longDictionarySupplier; @@ -210,6 +230,8 @@ public static VariantColumnAndIndexSupplier read( private final Supplier encodedValueColumnSupplier; @SuppressWarnings("unused") private final GenericIndexed valueIndexes; + @SuppressWarnings("unused") + private final GenericIndexed arrayElementIndexes; private final ImmutableBitmap nullValueBitmap; public VariantColumnAndIndexSupplier( @@ -222,7 +244,8 @@ public VariantColumnAndIndexSupplier( Supplier arrayDictionarySupplier, Supplier encodedValueColumnSupplier, GenericIndexed valueIndexes, - @SuppressWarnings("unused") BitmapFactory bitmapFactory, + GenericIndexed elementIndexes, + BitmapFactory bitmapFactory, @SuppressWarnings("unused") ColumnConfig columnConfig, @SuppressWarnings("unused") int numRows ) @@ -236,9 +259,17 @@ public VariantColumnAndIndexSupplier( this.arrayDictionarySupplier = arrayDictionarySupplier; this.encodedValueColumnSupplier = encodedValueColumnSupplier; this.valueIndexes = valueIndexes; + this.arrayElementIndexes = elementIndexes; + this.bitmapFactory = bitmapFactory; this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0); } + @Nullable + public Byte getVariantTypeSetByte() + { + return variantTypeSetByte; + } + @Override public NestedCommonFormatColumn get() { @@ -273,8 +304,98 @@ public T as(Class clazz) if (clazz.equals(NullValueIndex.class)) { final BitmapColumnIndex nullIndex = new SimpleImmutableBitmapIndex(nullValueBitmap); return (T) (NullValueIndex) () -> nullIndex; + } else if (clazz.equals(ValueIndexes.class) && variantTypeSetByte == null && logicalType.isArray()) { + return (T) new ArrayValueIndexes(); } - // coming soon... return null; } + + private ImmutableBitmap getBitmap(int idx) + { + if (idx < 0) { + return bitmapFactory.makeEmptyImmutableBitmap(); + } + + final ImmutableBitmap bitmap = valueIndexes.get(idx); + return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap; + } + + private class ArrayValueIndexes implements ValueIndexes + { + @Nullable + @Override + public BitmapColumnIndex forValue(Object value, TypeSignature valueType) + { + final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value) + .castTo(ExpressionType.fromColumnTypeStrict(logicalType)); + if (eval.value() == null) { + return null; + } + final Object[] arrayToMatch = eval.asArray(); + Indexed elements; + switch (logicalType.getElementType().getType()) { + case STRING: + elements = frontCodedStringDictionarySupplier != null + ? frontCodedStringDictionarySupplier.get() + : stringDictionary.singleThreaded(); + break; + case LONG: + elements = longDictionarySupplier.get(); + break; + case DOUBLE: + elements = doubleDictionarySupplier.get(); + break; + default: + throw DruidException.defensive( + "Unhandled array type [%s] how did this happen?", + logicalType.getElementType() + ); + } + + final int[] ids = new int[arrayToMatch.length]; + boolean hasMissingElement = false; + for (int i = 0; i < arrayToMatch.length; i++) { + if (logicalType.getElementType().is(ValueType.STRING)) { + ids[i] = elements.indexOf(StringUtils.toUtf8ByteBuffer((String) arrayToMatch[i])); + } else { + ids[i] = elements.indexOf(arrayToMatch[i]); + } + if (ids[i] < 0) { + hasMissingElement = true; + break; + } + } + + final boolean noMatch = hasMissingElement; + final FrontCodedIntArrayIndexed dictionary = arrayDictionarySupplier.get(); + return new SimpleBitmapColumnIndex() + { + @Override + public double estimateSelectivity(int totalRows) + { + if (noMatch) { + return 0.0; + } + final int id = dictionary.indexOf(ids); + if (id < 0) { + return 0.0; + } + return (double) getBitmap(id).size() / totalRows; + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + { + if (noMatch) { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + final int id = dictionary.indexOf(ids); + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); + } + return bitmapResultFactory.wrapDimensionValue(getBitmap(id)); + } + }; + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java index 9b107b1d78bd..001ca4dbfa06 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -29,6 +29,7 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.BitmapSerdeFactory; +import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.nested.NestedCommonFormatColumn; import org.apache.druid.segment.nested.NestedCommonFormatColumnSerializer; import org.apache.druid.segment.nested.NestedDataColumnSupplier; @@ -194,7 +195,6 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); } } @@ -218,7 +218,6 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); } } @@ -242,7 +241,6 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setNestedCommonFormatColumnSupplier(supplier); builder.setIndexSupplier(supplier, true, false); builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); } } @@ -260,13 +258,20 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo columnConfig ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); - capabilitiesBuilder.setDictionaryEncoded(true); - capabilitiesBuilder.setDictionaryValuesSorted(true); - capabilitiesBuilder.setDictionaryValuesUnique(true); + // if we are a mixed type, don't call ourself dictionary encoded for now so we don't end up doing the wrong thing + // in places. technically we could probably get by by indicating that our dictionary ids are not unique/sorted + // but just in case that still causes problems, skip it all... + if (supplier.getVariantTypeSetByte() == null) { + capabilitiesBuilder.setDictionaryEncoded(true); + capabilitiesBuilder.setDictionaryValuesSorted(true); + capabilitiesBuilder.setDictionaryValuesUnique(true); + } builder.setType(logicalType); builder.setNestedCommonFormatColumnSupplier(supplier); - builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, capabilitiesBuilder.hasNulls().isTrue())); - builder.setFilterable(true); + builder.setColumnFormat(new NestedCommonFormatColumn.Format( + logicalType, + capabilitiesBuilder.hasNulls().isTrue() + )); } } @@ -293,17 +298,16 @@ public void read(ByteBuffer buffer, ColumnBuilder builder, ColumnConfig columnCo builder.setType(logicalType); builder.setNestedCommonFormatColumnSupplier(supplier); builder.setColumnFormat(new NestedCommonFormatColumn.Format(logicalType, hasNulls)); - builder.setFilterable(true); } } public static class SerializerBuilder { - private ColumnType logicalType; + private ColumnType logicalType = ColumnType.NESTED_DATA; private boolean hasNulls; private boolean isVariantType; private ByteOrder byteOrder = ByteOrder.nativeOrder(); - BitmapSerdeFactory bitmapSerdeFactory; + BitmapSerdeFactory bitmapSerdeFactory = RoaringBitmapSerdeFactory.getInstance(); @Nullable private Serializer serializer = null; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java index 95df716b2ce5..ea8af5e85416 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NullColumnPartSerde.java @@ -108,7 +108,6 @@ public Deserializer getDeserializer() return (buffer, builder, columnConfig) -> { builder.setHasMultipleValues(false) .setHasNulls(true) - .setFilterable(true) // this is a bit sneaky, we set supplier to null here to act like a null column instead of a column // without any indexes, which is the default state .setIndexSupplier(null, true, false) diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NullValueIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/NullValueIndexSupplier.java index d66d15ca4b8e..476d88a1ce00 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NullValueIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NullValueIndexSupplier.java @@ -20,10 +20,10 @@ package org.apache.druid.segment.serde; import org.apache.druid.collections.bitmap.ImmutableBitmap; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.index.semantic.NullValueIndex; import javax.annotation.Nullable; @@ -55,7 +55,7 @@ public T as(Class clazz) private final class NullableNumericNullValueIndex implements NullValueIndex { @Override - public BitmapColumnIndex forNull() + public BitmapColumnIndex get() { return nullValueIndex; } diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java index de9c791c7eb7..e80ca2595a36 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java @@ -24,24 +24,26 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.spatial.ImmutableRTree; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.IndexedStringDictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.IndexedStringDruidPredicateIndex; -import org.apache.druid.segment.column.IndexedUtf8LexicographicalRangeIndex; -import org.apache.druid.segment.column.IndexedUtf8ValueSetIndex; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIndex; -import org.apache.druid.segment.column.SpatialIndex; import org.apache.druid.segment.column.StringEncodingStrategies; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.Indexed; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.IndexedStringDictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.IndexedStringDruidPredicateIndexes; +import org.apache.druid.segment.index.IndexedUtf8LexicographicalRangeIndexes; +import org.apache.druid.segment.index.IndexedUtf8ValueIndexes; +import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.SpatialIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; +import org.apache.druid.segment.index.semantic.Utf8ValueSetIndexes; +import org.apache.druid.segment.index.semantic.ValueIndexes; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -113,22 +115,26 @@ public T as(Class clazz) nullIndex = new SimpleImmutableBitmapIndex(bitmapFactory.makeEmptyImmutableBitmap()); } return (T) (NullValueIndex) () -> nullIndex; - } else if (clazz.equals(StringValueSetIndex.class)) { - return (T) new IndexedUtf8ValueSetIndex<>( + } else if ( + clazz.equals(StringValueSetIndexes.class) || + clazz.equals(Utf8ValueSetIndexes.class) || + clazz.equals(ValueIndexes.class) + ) { + return (T) new IndexedUtf8ValueIndexes<>( bitmapFactory, dict, singleThreadedBitmaps ); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new IndexedStringDruidPredicateIndex<>( + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new IndexedStringDruidPredicateIndexes<>( bitmapFactory, new StringEncodingStrategies.Utf8ToStringIndexed(dict), singleThreadedBitmaps, columnConfig, numRows ); - } else if (clazz.equals(LexicographicalRangeIndex.class)) { - return (T) new IndexedUtf8LexicographicalRangeIndex<>( + } else if (clazz.equals(LexicographicalRangeIndexes.class)) { + return (T) new IndexedUtf8LexicographicalRangeIndexes<>( bitmapFactory, dict, singleThreadedBitmaps, @@ -136,8 +142,10 @@ public T as(Class clazz) columnConfig, numRows ); - } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) - || clazz.equals(DictionaryEncodedValueIndex.class)) { + } else if ( + clazz.equals(DictionaryEncodedStringValueIndex.class) || + clazz.equals(DictionaryEncodedValueIndex.class) + ) { // Need string dictionary instead of UTF8 dictionary return (T) new IndexedStringDictionaryEncodedStringValueIndex<>( bitmapFactory, 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 345ecb93ac01..46bd682d33ce 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 @@ -300,7 +300,7 @@ public static DimensionSelector makeDimensionSelector( */ public static boolean canMapOverDictionary( final Expr.BindingAnalysis bindingAnalysis, - final ColumnCapabilities columnCapabilities + @Nullable final ColumnCapabilities columnCapabilities ) { Preconditions.checkState(bindingAnalysis.getRequiredBindings().size() == 1, "requiredBindings.size == 1"); @@ -321,7 +321,7 @@ public static Expr.ObjectBinding createBindings( ) { final List columns = plan.getAnalysis().getRequiredBindingsList(); - final Map suppliers = new HashMap<>(); + final Map> suppliers = new HashMap<>(); for (String columnName : columns) { final ColumnCapabilities capabilities = columnSelectorFactory.getColumnCapabilities(columnName); final boolean multiVal = capabilities != null && capabilities.hasMultipleValues().isTrue(); diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java index db72ec56d7f6..366b323bdefc 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java @@ -41,20 +41,20 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.IdMapping; import org.apache.druid.segment.VirtualColumn; -import org.apache.druid.segment.column.BitmapColumnIndex; 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.ColumnIndexSupplier; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.SimpleBitmapColumnIndex; -import org.apache.druid.segment.column.SimpleImmutableBitmapIterableIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.SimpleBitmapColumnIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import javax.annotation.Nullable; import java.util.Collections; @@ -226,12 +226,12 @@ public T as(Class clazz) if (clazz.equals(NullValueIndex.class)) { return (T) new ListFilteredNullValueIndex(underlyingIndex, idMapping, numRows); - } else if (clazz.equals(StringValueSetIndex.class)) { - return (T) new ListFilteredStringValueSetIndex(underlyingIndex, idMapping); - } else if (clazz.equals(DruidPredicateIndex.class)) { - return (T) new ListFilteredDruidPredicateIndex(underlyingIndex, idMapping); - } else if (clazz.equals(LexicographicalRangeIndex.class)) { - return (T) new ListFilteredLexicographicalRangeIndex(underlyingIndex, idMapping); + } else if (clazz.equals(StringValueSetIndexes.class)) { + return (T) new ListFilteredStringValueSetIndexes(underlyingIndex, idMapping); + } else if (clazz.equals(DruidPredicateIndexes.class)) { + return (T) new ListFilteredDruidPredicateIndexes(underlyingIndex, idMapping); + } else if (clazz.equals(LexicographicalRangeIndexes.class)) { + return (T) new ListFilteredLexicographicalRangeIndexes(underlyingIndex, idMapping); } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) || clazz.equals(DictionaryEncodedValueIndex.class)) { return (T) new ListFilteredDictionaryEncodedStringValueIndex(underlyingIndex, idMapping); } @@ -376,7 +376,7 @@ private ListFilteredNullValueIndex(DictionaryEncodedStringValueIndex delegate, I } @Override - public BitmapColumnIndex forNull() + public BitmapColumnIndex get() { return new SimpleImmutableBitmapIterableIndex() { @@ -405,11 +405,11 @@ protected Iterable getBitmapIterable() } } - private static class ListFilteredStringValueSetIndex extends BaseListFilteredColumnIndex - implements StringValueSetIndex + private static class ListFilteredStringValueSetIndexes extends BaseListFilteredColumnIndex + implements StringValueSetIndexes { - private ListFilteredStringValueSetIndex( + private ListFilteredStringValueSetIndexes( DictionaryEncodedStringValueIndex delegate, IdMapping idMapping ) @@ -494,11 +494,11 @@ private void findNext() } } - private static class ListFilteredDruidPredicateIndex extends BaseListFilteredColumnIndex - implements DruidPredicateIndex + private static class ListFilteredDruidPredicateIndexes extends BaseListFilteredColumnIndex + implements DruidPredicateIndexes { - private ListFilteredDruidPredicateIndex(DictionaryEncodedStringValueIndex delegate, IdMapping idMapping) + private ListFilteredDruidPredicateIndexes(DictionaryEncodedStringValueIndex delegate, IdMapping idMapping) { super(delegate, idMapping); } @@ -531,11 +531,11 @@ public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) } } - private static class ListFilteredLexicographicalRangeIndex extends BaseListFilteredColumnIndex - implements LexicographicalRangeIndex + private static class ListFilteredLexicographicalRangeIndexes extends BaseListFilteredColumnIndex + implements LexicographicalRangeIndexes { - private ListFilteredLexicographicalRangeIndex( + private ListFilteredLexicographicalRangeIndexes( DictionaryEncodedStringValueIndex delegate, IdMapping idMapping ) @@ -571,8 +571,9 @@ public BitmapColumnIndex forRange( public Iterable getBitmapIterable() { int startIndex, endIndex; + final int firstValue = NullHandling.isNullOrEquivalent(delegate.getValue(idMapping.getReverseId(0))) ? 1 : 0; if (startValue == null) { - startIndex = 0; + startIndex = firstValue; } else { final int found = getReverseIndex(NullHandling.emptyToNullIfNeeded(startValue)); if (found >= 0) { diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java index 02c3d879e837..dd28ea2eb5e2 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java @@ -1017,7 +1017,14 @@ private void computeVectorsIfNeeded() if (v instanceof Number) { l = ((Number) v).longValue(); } else { - l = GuavaUtils.tryParseLong(String.valueOf(v)); + final String s = String.valueOf(v); + l = GuavaUtils.tryParseLong(s); + if (l == null) { + final Double d = Doubles.tryParse(s); + if (d != null) { + l = d.longValue(); + } + } } if (l != null) { longVector[i] = l; diff --git a/processing/src/test/java/org/apache/druid/frame/testutil/RowReadingVectorColumnProcessorFactory.java b/processing/src/test/java/org/apache/druid/frame/testutil/RowReadingVectorColumnProcessorFactory.java index fcaec310e0ac..ad89346f2d68 100644 --- a/processing/src/test/java/org/apache/druid/frame/testutil/RowReadingVectorColumnProcessorFactory.java +++ b/processing/src/test/java/org/apache/druid/frame/testutil/RowReadingVectorColumnProcessorFactory.java @@ -127,6 +127,12 @@ public Supplier makeLongProcessor(ColumnCapabilities capabilities, Vec }; } + @Override + public Supplier makeArrayProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) + { + return selector::getObjectVector; + } + @Override public Supplier makeObjectProcessor(ColumnCapabilities capabilities, VectorObjectSelector selector) { diff --git a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java index 1171e0f5098e..45191ade72ec 100644 --- a/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java +++ b/processing/src/test/java/org/apache/druid/math/expr/EvalTest.java @@ -987,6 +987,48 @@ public void testBooleanInputs() } } + @Test + public void testArrayComparison() + { + Expr.ObjectBinding bindings = InputBindings.forInputSuppliers( + ImmutableMap.>builder() + .put( + "stringArray", + InputBindings.inputSupplier(ExpressionType.STRING_ARRAY, () -> new Object[]{"a", "b", null, "c"}) + ) + .put( + "longArray", + InputBindings.inputSupplier(ExpressionType.LONG_ARRAY, () -> new Object[]{1L, null, 2L, 3L}) + ) + .put( + "doubleArray", + InputBindings.inputSupplier(ExpressionType.DOUBLE_ARRAY, () -> new Object[]{1.1, 2.2, 3.3, null}) + ) + .build() + ); + + Assert.assertEquals(0L, eval("['a','b',null,'c'] > stringArray", bindings).value()); + Assert.assertEquals(1L, eval("['a','b',null,'c'] >= stringArray", bindings).value()); + Assert.assertEquals(1L, eval("['a','b',null,'c'] == stringArray", bindings).value()); + Assert.assertEquals(0L, eval("['a','b',null,'c'] != stringArray", bindings).value()); + Assert.assertEquals(1L, eval("['a','b',null,'c'] <= stringArray", bindings).value()); + Assert.assertEquals(0L, eval("['a','b',null,'c'] < stringArray", bindings).value()); + + Assert.assertEquals(0L, eval("[1,null,2,3] > longArray", bindings).value()); + Assert.assertEquals(1L, eval("[1,null,2,3] >= longArray", bindings).value()); + Assert.assertEquals(1L, eval("[1,null,2,3] == longArray", bindings).value()); + Assert.assertEquals(0L, eval("[1,null,2,3] != longArray", bindings).value()); + Assert.assertEquals(1L, eval("[1,null,2,3] <= longArray", bindings).value()); + Assert.assertEquals(0L, eval("[1,null,2,3] < longArray", bindings).value()); + + Assert.assertEquals(0L, eval("[1.1,2.2,3.3,null] > doubleArray", bindings).value()); + Assert.assertEquals(1L, eval("[1.1,2.2,3.3,null] >= doubleArray", bindings).value()); + Assert.assertEquals(1L, eval("[1.1,2.2,3.3,null] == doubleArray", bindings).value()); + Assert.assertEquals(0L, eval("[1.1,2.2,3.3,null] != doubleArray", bindings).value()); + Assert.assertEquals(1L, eval("[1.1,2.2,3.3,null] <= doubleArray", bindings).value()); + Assert.assertEquals(0L, eval("[1.1,2.2,3.3,null] < doubleArray", bindings).value()); + } + @Test public void testValueOrDefault() { diff --git a/processing/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java index 4b70cec0e0b4..fb38e803ebaa 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/ExprMacroTest.java @@ -37,7 +37,7 @@ public class ExprMacroTest private static final String IPV4_STRING = "192.168.0.1"; private static final long IPV4_LONG = 3232235521L; private static final Expr.ObjectBinding BINDINGS = InputBindings.forInputSuppliers( - ImmutableMap.builder() + ImmutableMap.>builder() .put("t", InputBindings.inputSupplier( ExpressionType.LONG, diff --git a/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java index cb641c59e0d2..7109f51f44e6 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/HyperUniqueExpressionsTest.java @@ -52,7 +52,7 @@ public class HyperUniqueExpressionsTest extends InitializedNullHandlingTest private static final double SOME_DOUBLE = 1.234; Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers( - new ImmutableMap.Builder() + new ImmutableMap.Builder>() .put("hll", InputBindings.inputSupplier(HyperUniqueExpressions.TYPE, HyperLogLogCollector::makeLatestCollector)) .put("string", InputBindings.inputSupplier(ExpressionType.STRING, () -> SOME_STRING)) .put("long", InputBindings.inputSupplier(ExpressionType.LONG, () -> SOME_LONG)) diff --git a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java index 06726289ce69..fae95259d4ad 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/NestedDataExpressionsTest.java @@ -81,7 +81,7 @@ public class NestedDataExpressionsTest extends InitializedNullHandlingTest ); Expr.ObjectBinding inputBindings = InputBindings.forInputSuppliers( - new ImmutableMap.Builder() + new ImmutableMap.Builder>() .put("nest", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> NEST)) .put("nestWrapped", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> new StructuredData(NEST))) .put("nester", InputBindings.inputSupplier(ExpressionType.NESTED_DATA, () -> NESTER)) diff --git a/processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java b/processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java index f0e3f3c843d7..3f3ce780c8c9 100644 --- a/processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java +++ b/processing/src/test/java/org/apache/druid/query/expression/RegexpReplaceExprMacroTest.java @@ -223,7 +223,7 @@ public void testUrlIdReplacementDynamic() "regexp_replace(regexp_replace(a, pattern1, replacement1), pattern2, replacement2)", InputBindings.forInputSuppliers( ImmutableMap - .builder() + .>builder() .put("a", InputBindings.inputSupplier(ExpressionType.STRING, () -> "http://example.com/path/to?query")) .put("pattern1", InputBindings.inputSupplier(ExpressionType.STRING, () -> "\\?(.*)$")) .put("pattern2", InputBindings.inputSupplier(ExpressionType.STRING, () -> "/(\\w+)(?=/|$)")) diff --git a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java index cf47b2243a13..41f2480621da 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java @@ -31,12 +31,12 @@ import org.apache.druid.query.extraction.RegexDimExtractionFn; import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowBasedColumnSelectorFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.column.StringValueSetIndex; -import org.apache.druid.segment.column.Utf8ValueSetIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; +import org.apache.druid.segment.index.semantic.Utf8ValueSetIndexes; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -260,15 +260,15 @@ public void testUsesUtf8SetIndex() final ColumnIndexSelector indexSelector = Mockito.mock(ColumnIndexSelector.class); final ColumnIndexSupplier indexSupplier = Mockito.mock(ColumnIndexSupplier.class); - final Utf8ValueSetIndex valueIndex = Mockito.mock(Utf8ValueSetIndex.class); + final Utf8ValueSetIndexes valueIndexes = Mockito.mock(Utf8ValueSetIndexes.class); final BitmapColumnIndex bitmapColumnIndex = Mockito.mock(BitmapColumnIndex.class); final InDimFilter.ValuesSet expectedValuesSet = new InDimFilter.ValuesSet(); expectedValuesSet.addAll(Arrays.asList("v1", "v2")); Mockito.when(indexSelector.getIndexSupplier("dim0")).thenReturn(indexSupplier); - Mockito.when(indexSupplier.as(Utf8ValueSetIndex.class)).thenReturn(valueIndex); - Mockito.when(valueIndex.forSortedValuesUtf8(expectedValuesSet.toUtf8())).thenReturn(bitmapColumnIndex); + Mockito.when(indexSupplier.as(Utf8ValueSetIndexes.class)).thenReturn(valueIndexes); + Mockito.when(valueIndexes.forSortedValuesUtf8(expectedValuesSet.toUtf8())).thenReturn(bitmapColumnIndex); final BitmapColumnIndex retVal = inFilter.getBitmapColumnIndex(indexSelector); Assert.assertSame("inFilter returns the intended bitmapColumnIndex", bitmapColumnIndex, retVal); @@ -284,15 +284,15 @@ public void testUsesStringSetIndex() final ColumnIndexSelector indexSelector = Mockito.mock(ColumnIndexSelector.class); final ColumnIndexSupplier indexSupplier = Mockito.mock(ColumnIndexSupplier.class); - final StringValueSetIndex valueIndex = Mockito.mock(StringValueSetIndex.class); + final StringValueSetIndexes valueIndex = Mockito.mock(StringValueSetIndexes.class); final BitmapColumnIndex bitmapColumnIndex = Mockito.mock(BitmapColumnIndex.class); final InDimFilter.ValuesSet expectedValuesSet = new InDimFilter.ValuesSet(); expectedValuesSet.addAll(Arrays.asList("v1", "v2")); Mockito.when(indexSelector.getIndexSupplier("dim0")).thenReturn(indexSupplier); - Mockito.when(indexSupplier.as(Utf8ValueSetIndex.class)).thenReturn(null); // Will check for UTF-8 first. - Mockito.when(indexSupplier.as(StringValueSetIndex.class)).thenReturn(valueIndex); + Mockito.when(indexSupplier.as(Utf8ValueSetIndexes.class)).thenReturn(null); // Will check for UTF-8 first. + Mockito.when(indexSupplier.as(StringValueSetIndexes.class)).thenReturn(valueIndex); Mockito.when(valueIndex.forSortedValues(expectedValuesSet)).thenReturn(bitmapColumnIndex); final BitmapColumnIndex retVal = inFilter.getBitmapColumnIndex(indexSelector); diff --git a/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java index c51ec817b739..f0018b94f7cf 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java @@ -24,10 +24,10 @@ import nl.jqno.equalsverifier.EqualsVerifier; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.query.extraction.SubstringDimExtractionFn; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Rule; @@ -111,11 +111,11 @@ public void testPrefixMatchUsesRangeIndex() final ColumnIndexSelector indexSelector = Mockito.mock(ColumnIndexSelector.class); final ColumnIndexSupplier indexSupplier = Mockito.mock(ColumnIndexSupplier.class); - final LexicographicalRangeIndex rangeIndex = Mockito.mock(LexicographicalRangeIndex.class); + final LexicographicalRangeIndexes rangeIndex = Mockito.mock(LexicographicalRangeIndexes.class); final BitmapColumnIndex bitmapColumnIndex = Mockito.mock(BitmapColumnIndex.class); Mockito.when(indexSelector.getIndexSupplier("dim0")).thenReturn(indexSupplier); - Mockito.when(indexSupplier.as(LexicographicalRangeIndex.class)).thenReturn(rangeIndex); + Mockito.when(indexSupplier.as(LexicographicalRangeIndexes.class)).thenReturn(rangeIndex); Mockito.when( // Verify that likeFilter uses forRange without a matcher predicate; it's unnecessary and slows things down rangeIndex.forRange("f", false, "f" + Character.MAX_VALUE, false) @@ -135,11 +135,11 @@ public void testExactMatchUsesValueIndex() final ColumnIndexSelector indexSelector = Mockito.mock(ColumnIndexSelector.class); final ColumnIndexSupplier indexSupplier = Mockito.mock(ColumnIndexSupplier.class); - final StringValueSetIndex valueIndex = Mockito.mock(StringValueSetIndex.class); + final StringValueSetIndexes valueIndex = Mockito.mock(StringValueSetIndexes.class); final BitmapColumnIndex bitmapColumnIndex = Mockito.mock(BitmapColumnIndex.class); Mockito.when(indexSelector.getIndexSupplier("dim0")).thenReturn(indexSupplier); - Mockito.when(indexSupplier.as(StringValueSetIndex.class)).thenReturn(valueIndex); + Mockito.when(indexSupplier.as(StringValueSetIndexes.class)).thenReturn(valueIndex); Mockito.when(valueIndex.forValue("f")).thenReturn(bitmapColumnIndex); final BitmapColumnIndex retVal = likeFilter.getBitmapColumnIndex(indexSelector); diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 2f510cae56b8..6133a16b1eaa 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -789,7 +789,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryTypeGauntlet() throw + "[1672531200000, null, 0, 0.0, true, 51, 1, [], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, null, 2, 0.0, false, b, b, 2, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, a, 1, 1.0, true, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " - + "[1672531200000, b, 4, 3.3, true, 4, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + + "[1672531200000, b, 4, 3.3, true, 1, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, c, 0, 4.4, true, hello, {}, [], {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, d, 5, 5.9, false, null, a, 6, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]" @@ -802,7 +802,7 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryTypeGauntlet() throw + "[1672531200000, null, null, null, true, 51, 1, [], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [true, false, true], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, , 2, null, false, b, b, 2, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [true, true], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, a, 1, 1.0, true, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [true, false, true], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " - + "[1672531200000, b, 4, 3.3, true, 4, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + + "[1672531200000, b, 4, 3.3, true, 1, {}, 4, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, false, true], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, c, null, 4.4, true, hello, {}, [], {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [false], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, d, 5, 5.9, false, null, a, 6, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], " + "[1672531200000, null, 3, 2.0, null, 3.0, 3.3, 3, {a=300}, {x=4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [true, null, true], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]" diff --git a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java index 05a91cd658f1..fe8900c0dd8f 100644 --- a/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/AutoTypeColumnIndexerTest.java @@ -482,45 +482,36 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals(2L, valueSelector.getObject()); Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(3.3), valueSelector.getObject()); + dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals(3.3, valueSelector.getObject()); Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("3.3", dimensionSelector.getObject()); + columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); + dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); + dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java index 9d71c0c1e371..e042fd39b74f 100644 --- a/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java @@ -22,14 +22,14 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.DefaultBitmapResultFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringValueSetIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.serde.NoIndexesColumnIndexSupplier; import org.easymock.EasyMock; import org.junit.Assert; @@ -73,8 +73,8 @@ public void setup() ).anyTimes(); EasyMock.expect(holder.getColumn()).andReturn(stringColumn).anyTimes(); EasyMock.expect(holder.getIndexSupplier()).andReturn(indexSupplier).anyTimes(); - StringValueSetIndex someIndex = EasyMock.createMock(StringValueSetIndex.class); - EasyMock.expect(indexSupplier.as(StringValueSetIndex.class)).andReturn(someIndex).anyTimes(); + StringValueSetIndexes someIndex = EasyMock.createMock(StringValueSetIndexes.class); + EasyMock.expect(indexSupplier.as(StringValueSetIndexes.class)).andReturn(someIndex).anyTimes(); DictionaryEncodedStringValueIndex valueIndex = EasyMock.createMock(DictionaryEncodedStringValueIndex.class); EasyMock.expect(indexSupplier.as(DictionaryEncodedStringValueIndex.class)).andReturn(valueIndex).anyTimes(); BitmapColumnIndex columnIndex = EasyMock.createMock(BitmapColumnIndex.class); @@ -94,7 +94,6 @@ public void setup() .setDictionaryValuesUnique(true) .setDictionaryValuesSorted(true) .setHasBitmapIndexes(true) - .setFilterable(true) ).anyTimes(); EasyMock.replay(bitmapFactory, virtualColumns, index, indexSupplier, holder, stringColumn, nonStringHolder, someIndex, columnIndex, valueIndex, someBitmap); @@ -109,7 +108,7 @@ public void testStringDictionaryUseIndex() ); Assert.assertNotNull(bitmapIndex); - StringValueSetIndex valueIndex = supplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueIndex = supplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueIndex); ImmutableBitmap valueBitmap = valueIndex.forValue("foo") .computeBitmapResult( @@ -128,7 +127,7 @@ public void testNonStringDictionaryDoNotUseIndex() ); Assert.assertNull(bitmapIndex); - StringValueSetIndex valueIndex = supplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueIndex = supplier.as(StringValueSetIndexes.class); Assert.assertNull(valueIndex); EasyMock.verify(bitmapFactory, virtualColumns, index, indexSupplier); } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 6425e12def4d..664a28603cb9 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -318,7 +318,7 @@ private QueryableIndex mergeIndexes( i++; } else { persisted.add( - TestHelper.getTestIndexIO().loadIndex( + indexIO.loadIndex( indexMerger.persist( incrementalIndex, new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID().toString())), @@ -336,7 +336,7 @@ private QueryableIndex mergeIndexes( } if (i != 0) { persisted.add( - TestHelper.getTestIndexIO().loadIndex( + indexIO.loadIndex( indexMerger.persist( incrementalIndex, new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID().toString())), @@ -347,7 +347,7 @@ private QueryableIndex mergeIndexes( ); } - final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex( + final QueryableIndex merged = indexIO.loadIndex( indexMerger.mergeQueryableIndex( persisted, true, diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java index 9f0ee07d0928..3b3706c6ea5a 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java @@ -32,11 +32,11 @@ import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.DictionaryEncodedColumn; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.IncrementalIndexTest; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.incremental.IncrementalIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.junit.Assert; import org.junit.Before; @@ -185,8 +185,8 @@ public void testStringColumnNullHandling() throws Exception final DictionaryEncodedStringValueIndex valueIndex = columnHolder.getIndexSupplier().as( DictionaryEncodedStringValueIndex.class ); - final StringValueSetIndex valueSetIndex = columnHolder.getIndexSupplier().as( - StringValueSetIndex.class + final StringValueSetIndexes valueSetIndex = columnHolder.getIndexSupplier().as( + StringValueSetIndexes.class ); // Read through the column to find all the rows that should match null. diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 30c41bea70f5..5cc10619a72e 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -49,7 +49,6 @@ import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.BitmapValues; import org.apache.druid.segment.data.CompressionFactory; @@ -61,6 +60,7 @@ import org.apache.druid.segment.incremental.IncrementalIndexAdapter; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.OnheapIncrementalIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.Interval; @@ -127,7 +127,7 @@ static BitmapValues getBitmapIndex(QueryableIndexIndexableAdapter adapter, Strin return BitmapValues.EMPTY; } - final StringValueSetIndex index = indexSupplier.as(StringValueSetIndex.class); + final StringValueSetIndexes index = indexSupplier.as(StringValueSetIndexes.class); if (index == null) { return BitmapValues.EMPTY; } diff --git a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java index a50df7114982..3c8b376dbcad 100644 --- a/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java +++ b/processing/src/test/java/org/apache/druid/segment/NestedDataColumnIndexerTest.java @@ -481,45 +481,35 @@ public void testNestedColumnIndexerSchemaDiscoveryRootVariant() throws IndexSize ColumnSelectorFactory columnSelectorFactory = cursorList.get(0).getColumnSelectorFactory(); ColumnValueSelector valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap("a"), valueSelector.getObject()); + DimensionSelector dimensionSelector = cursorList.get(0).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals("a", valueSelector.getObject()); + Assert.assertEquals("a", dimensionSelector.getObject()); columnSelectorFactory = cursorList.get(1).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(2L), valueSelector.getObject()); + dimensionSelector = cursorList.get(1).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals(2L, valueSelector.getObject()); Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("2", dimensionSelector.getObject()); columnSelectorFactory = cursorList.get(2).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); - Assert.assertEquals(StructuredData.wrap(3.3), valueSelector.getObject()); + dimensionSelector = cursorList.get(2).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); + Assert.assertEquals(3.3, valueSelector.getObject()); Assert.assertFalse(valueSelector.isNull()); + Assert.assertEquals("3.3", dimensionSelector.getObject()); columnSelectorFactory = cursorList.get(3).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); + dimensionSelector = cursorList.get(3).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); columnSelectorFactory = cursorList.get(4).getColumnSelectorFactory(); valueSelector = columnSelectorFactory.makeColumnValueSelector(VARIANT_COL); - Assert.assertThrows( - UnsupportedOperationException.class, - () -> cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec) - ); + dimensionSelector = cursorList.get(4).getColumnSelectorFactory().makeDimensionSelector(dimensionSpec); Assert.assertNull(valueSelector.getObject()); + Assert.assertNull(dimensionSelector.getObject()); } @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 cb2cb95d5df4..17ee7d6e60e5 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 @@ -37,8 +37,7 @@ public void testSerde() throws Exception .setHasMultipleValues(true) .setHasSpatialIndexes(true) .setType(ColumnType.UNKNOWN_COMPLEX) - .setHasNulls(true) - .setFilterable(true)); + .setHasNulls(true)); Assert.assertFalse(json.contains("filterable")); @@ -51,7 +50,6 @@ public void testSerde() throws Exception Assert.assertTrue(cc.hasBitmapIndexes()); // hasNulls and isFilterable are computed, these should not be set Assert.assertFalse(cc.hasNulls().isTrue()); - Assert.assertFalse(cc.isFilterable()); } @Test @@ -78,6 +76,5 @@ public void testDeserialization() throws Exception Assert.assertTrue(cc.hasBitmapIndexes()); // hasNulls and isFilterable are computed, these should not be set Assert.assertFalse(cc.hasNulls().isTrue()); - Assert.assertFalse(cc.isFilterable()); } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 3495d1cb6d9c..b1e31638877e 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -79,7 +79,6 @@ import org.apache.druid.segment.RowBasedStorageAdapter; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.StringEncodingStrategy; @@ -92,6 +91,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import org.apache.druid.segment.vector.VectorCursor; @@ -159,6 +159,9 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .add(new DoubleDimensionSchema("d0")) .add(new FloatDimensionSchema("f0")) .add(new LongDimensionSchema("l0")) + .add(new AutoTypeColumnSchema("arrayString")) + .add(new AutoTypeColumnSchema("arrayLong")) + .add(new AutoTypeColumnSchema("arrayDouble")) .build() ); @@ -179,15 +182,84 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .add("d0", ColumnType.DOUBLE) .add("f0", ColumnType.FLOAT) .add("l0", ColumnType.LONG) + .add("arrayString", ColumnType.STRING_ARRAY) + .add("arrayLong", ColumnType.LONG_ARRAY) + .add("arrayDouble", ColumnType.DOUBLE_ARRAY) .build(); static final List DEFAULT_ROWS = ImmutableList.of( - makeDefaultSchemaRow("0", "", ImmutableList.of("a", "b"), "2017-07-25", 0.0, 0.0f, 0L), - makeDefaultSchemaRow("1", "10", ImmutableList.of(), "2017-07-25", 10.1, 10.1f, 100L), - makeDefaultSchemaRow("2", "2", ImmutableList.of(""), "2017-05-25", null, 5.5f, 40L), - makeDefaultSchemaRow("3", "1", ImmutableList.of("a"), "2020-01-25", 120.0245, 110.0f, null), - makeDefaultSchemaRow("4", "abdef", ImmutableList.of("c"), null, 60.0, null, 9001L), - makeDefaultSchemaRow("5", "abc", null, "2020-01-25", 765.432, 123.45f, 12345L) + makeDefaultSchemaRow( + "0", + "", + ImmutableList.of("a", "b"), + "2017-07-25", + 0.0, + 0.0f, + 0L, + ImmutableList.of("a", "b", "c"), + ImmutableList.of(1L, 2L, 3L), + ImmutableList.of(1.1, 2.2, 3.3) + ), + makeDefaultSchemaRow( + "1", + "10", + ImmutableList.of(), + "2017-07-25", + 10.1, + 10.1f, + 100L, + ImmutableList.of(), + ImmutableList.of(), + new Object[]{1.1, 2.2, 3.3} + ), + makeDefaultSchemaRow( + "2", + "2", + ImmutableList.of(""), + "2017-05-25", + null, + 5.5f, + 40L, + null, + new Object[]{1L, 2L, 3L}, + Collections.singletonList(null) + ), + makeDefaultSchemaRow( + "3", + "1", + ImmutableList.of("a"), + "2020-01-25", + 120.0245, + 110.0f, + null, + new Object[]{"a", "b", "c"}, + null, + ImmutableList.of() + ), + makeDefaultSchemaRow( + "4", + "abdef", + ImmutableList.of("c"), + null, + 60.0, + null, + 9001L, + ImmutableList.of("c", "d"), + Collections.singletonList(null), + new Object[]{-1.1, -333.3} + ), + makeDefaultSchemaRow( + "5", + "abc", + null, + "2020-01-25", + 765.432, + 123.45f, + 12345L, + Collections.singletonList(null), + new Object[]{123L, 345L}, + null + ) ); static final IncrementalIndexSchema DEFAULT_INDEX_SCHEMA = new IncrementalIndexSchema.Builder() @@ -209,12 +281,15 @@ static InputRow makeSchemaRow( @Nullable Object... elements ) { - Preconditions.checkArgument(signature.size() == elements.length); Map mapRow = Maps.newHashMapWithExpectedSize(signature.size()); for (int i = 0; i < signature.size(); i++) { final String columnName = signature.getColumnName(i); - final Object value = elements[i]; - mapRow.put(columnName, value); + if (elements != null && i < elements.length) { + final Object value = elements[i]; + mapRow.put(columnName, value); + } else { + mapRow.put(columnName, null); + } } return parser.parseBatch(mapRow).get(0); } @@ -328,6 +403,34 @@ public static Collection makeConstructors() return Pair.of(new IncrementalIndexStorageAdapter(index), index); } ) + .put( + "incrementalAutoTypes", + input -> { + input.indexSpec(IndexSpec.builder().build()); + input.mapSchema( + schema -> + new IncrementalIndexSchema( + schema.getMinTimestamp(), + schema.getTimestampSpec(), + schema.getGran(), + schema.getVirtualColumns(), + schema.getDimensionsSpec().withDimensions( + schema.getDimensionsSpec() + .getDimensions() + .stream() + .map( + dimensionSchema -> new AutoTypeColumnSchema(dimensionSchema.getName()) + ) + .collect(Collectors.toList()) + ), + schema.getMetrics(), + schema.isRollup() + ) + ); + final IncrementalIndex index = input.buildIncrementalIndex(); + return Pair.of(new IncrementalIndexStorageAdapter(index), index); + } + ) .put( "mmappedAutoTypes", input -> { @@ -439,10 +542,48 @@ public static Collection makeConstructors() input -> Pair.of(input.buildRowBasedSegmentWithTypeSignature().asStorageAdapter(), () -> {}) ) .put("frame (row-based)", input -> { + // remove array type columns from frames since they aren't currently supported other than string + input.mapSchema( + schema -> + new IncrementalIndexSchema( + schema.getMinTimestamp(), + schema.getTimestampSpec(), + schema.getGran(), + schema.getVirtualColumns(), + schema.getDimensionsSpec().withDimensions( + schema.getDimensionsSpec() + .getDimensions() + .stream() + .filter(dimensionSchema -> !(dimensionSchema instanceof AutoTypeColumnSchema)) + .collect(Collectors.toList()) + ), + schema.getMetrics(), + schema.isRollup() + ) + ); final FrameSegment segment = input.buildFrameSegment(FrameType.ROW_BASED); return Pair.of(segment.asStorageAdapter(), segment); }) .put("frame (columnar)", input -> { + // remove array type columns from frames since they aren't currently supported other than string + input.mapSchema( + schema -> + new IncrementalIndexSchema( + schema.getMinTimestamp(), + schema.getTimestampSpec(), + schema.getGran(), + schema.getVirtualColumns(), + schema.getDimensionsSpec().withDimensions( + schema.getDimensionsSpec() + .getDimensions() + .stream() + .filter(dimensionSchema -> !(dimensionSchema instanceof AutoTypeColumnSchema)) + .collect(Collectors.toList()) + ), + schema.getMetrics(), + schema.isRollup() + ) + ); final FrameSegment segment = input.buildFrameSegment(FrameType.COLUMNAR); return Pair.of(segment.asStorageAdapter(), segment); }) @@ -491,6 +632,14 @@ public static Collection makeConstructors() return constructors; } + protected boolean isAutoSchema() + { + if (testName.contains("AutoTypes")) { + return true; + } + return false; + } + private Filter makeFilter(final DimFilter dimFilter) { if (dimFilter == null) { @@ -900,6 +1049,30 @@ protected void assertFilterMatches( assertFilterMatches(filter, expectedRows, testVectorized); } + protected void assertFilterMatchesSkipArrays( + final DimFilter filter, + final List expectedRows + ) + { + // IncrementalIndex, RowBasedSegment cannot vectorize. + // Columnar FrameStorageAdapter *can* vectorize, but the tests won't pass, because the vectorizable cases + // differ from QueryableIndexStorageAdapter due to frames not having indexes. So, skip these too. + final boolean testVectorized = + !(adapter instanceof IncrementalIndexStorageAdapter) + && !(adapter instanceof RowBasedStorageAdapter) + && !(adapter instanceof FrameStorageAdapter); + + if (isAutoSchema()) { + Throwable t = Assert.assertThrows( + Throwable.class, + () -> assertFilterMatches(filter, expectedRows, testVectorized) + ); + Assert.assertTrue(t.getMessage().contains("ARRAY")); + } else { + assertFilterMatches(filter, expectedRows, testVectorized); + } + } + protected void assertFilterMatchesSkipVectorize( final DimFilter filter, final List expectedRows @@ -914,66 +1087,58 @@ private void assertFilterMatches( final boolean testVectorized ) { - try { + Assert.assertEquals( + "Cursor: " + filter, + expectedRows, + selectColumnValuesMatchingFilter(filter, "dim0") + ); + + if (testVectorized) { Assert.assertEquals( - "Cursor: " + filter, + "Cursor (vectorized): " + filter, expectedRows, - selectColumnValuesMatchingFilter(filter, "dim0") + selectColumnValuesMatchingFilterUsingVectorCursor(filter, "dim0") ); - if (testVectorized) { - Assert.assertEquals( - "Cursor (vectorized): " + filter, - expectedRows, - selectColumnValuesMatchingFilterUsingVectorCursor(filter, "dim0") - ); - - Assert.assertEquals( - "Cursor Virtual Column (vectorized): " + filter, - expectedRows, - selectColumnValuesMatchingFilterUsingVectorVirtualColumnCursor(filter, "vdim0", "dim0") - ); - } - Assert.assertEquals( - "Cursor with postFiltering: " + filter, + "Cursor Virtual Column (vectorized): " + filter, expectedRows, - selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0") + selectColumnValuesMatchingFilterUsingVectorVirtualColumnCursor(filter, "vdim0", "dim0") ); + } - if (testVectorized) { - Assert.assertEquals( - "Cursor with postFiltering (vectorized): " + filter, - expectedRows, - selectColumnValuesMatchingFilterUsingVectorizedPostFiltering(filter, "dim0") - ); - } + Assert.assertEquals( + "Cursor with postFiltering: " + filter, + expectedRows, + selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0") + ); + if (testVectorized) { Assert.assertEquals( - "Filtered aggregator: " + filter, - expectedRows.size(), - selectCountUsingFilteredAggregator(filter) + "Cursor with postFiltering (vectorized): " + filter, + expectedRows, + selectColumnValuesMatchingFilterUsingVectorizedPostFiltering(filter, "dim0") ); + } - if (testVectorized) { - Assert.assertEquals( - "Filtered aggregator (vectorized): " + filter, - expectedRows.size(), - selectCountUsingVectorizedFilteredAggregator(filter) - ); - } + Assert.assertEquals( + "Filtered aggregator: " + filter, + expectedRows.size(), + selectCountUsingFilteredAggregator(filter) + ); + if (testVectorized) { Assert.assertEquals( - "RowBasedColumnSelectorFactory: " + filter, - expectedRows, - selectColumnValuesMatchingFilterUsingRowBasedColumnSelectorFactory(filter, "dim0") + "Filtered aggregator (vectorized): " + filter, + expectedRows.size(), + selectCountUsingVectorizedFilteredAggregator(filter) ); } - catch (ISE ise) { - // ignore failures resulting from 'auto' - if (!(testName.contains("AutoTypes") && "Unsupported type[ARRAY]".equals(ise.getMessage()))) { - throw ise; - } - } + + Assert.assertEquals( + "RowBasedColumnSelectorFactory: " + filter, + expectedRows, + selectColumnValuesMatchingFilterUsingRowBasedColumnSelectorFactory(filter, "dim0") + ); } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java index ec952c31d100..933843abb257 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BoundFilterTest.java @@ -90,7 +90,11 @@ public void testLexicographicMatchEverything() ); for (BoundDimFilter filter : filters) { - assertFilterMatches(filter, ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); + if (filter.getDimension().equals("dim2")) { + assertFilterMatchesSkipArrays(filter, ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); + } else { + assertFilterMatches(filter, ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); + } } } @@ -105,12 +109,16 @@ public void testLexicographicMatchWithEmptyString() ); if (NullHandling.replaceWithDefault()) { for (BoundDimFilter filter : filters) { - assertFilterMatches(filter, ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); + if (filter.getDimension().equals("dim2")) { + assertFilterMatchesSkipArrays(filter, ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); + } else { + assertFilterMatches(filter, ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); + } } } else { assertFilterMatches(filters.get(0), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); assertFilterMatches(filters.get(1), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")); - assertFilterMatches(filters.get(2), ImmutableList.of("0", "2", "3", "4", "6", "7")); + assertFilterMatchesSkipArrays(filters.get(2), ImmutableList.of("0", "2", "3", "4", "6", "7")); assertFilterMatches(filters.get(3), ImmutableList.of()); } } @@ -127,12 +135,12 @@ public void testLexicographicMatchNull() ImmutableList.of("0") ); if (NullHandling.replaceWithDefault()) { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "", "", false, false, false, null, StringComparators.LEXICOGRAPHIC), ImmutableList.of("1", "2", "5") ); } else { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "", "", false, false, false, null, StringComparators.LEXICOGRAPHIC), ImmutableList.of("2") ); @@ -278,7 +286,7 @@ public void testAlphaNumericMatchNull() ImmutableList.of("0") ); if (NullHandling.replaceWithDefault()) { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "", "", false, false, true, null, StringComparators.ALPHANUMERIC), ImmutableList.of("1", "2", "5") ); @@ -287,7 +295,7 @@ public void testAlphaNumericMatchNull() ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") ); } else { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "", "", false, false, true, null, StringComparators.ALPHANUMERIC), ImmutableList.of("2") ); @@ -387,7 +395,7 @@ public void testNumericMatchNull() ImmutableList.of("0") ); if (NullHandling.replaceWithDefault()) { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "", "", false, false, false, null, StringComparators.NUMERIC), ImmutableList.of("1", "2", "5") ); @@ -396,7 +404,7 @@ public void testNumericMatchNull() ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") ); } else { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "", "", false, false, false, null, StringComparators.NUMERIC), ImmutableList.of("2") ); @@ -470,6 +478,10 @@ public void testListFilteredVirtualColumn() ImmutableList.of("0", "1", "2", "3", "4", "5", "6") ); + if (isAutoSchema()) { + // bail out, auto ingests arrays instead of mvds and this virtual column is for mvd stuff + return; + } assertFilterMatchesSkipVectorize( new BoundDimFilter("allow-dim2", "a", "c", false, false, false, null, StringComparators.LEXICOGRAPHIC), ImmutableList.of("0", "3", "6") @@ -594,7 +606,7 @@ public void testMatchWithExtractionFn() ImmutableList.of("1", "2", "3") ); - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter( "dim2", "super-", @@ -609,7 +621,7 @@ public void testMatchWithExtractionFn() ); if (NullHandling.replaceWithDefault()) { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter( "dim2", "super-null", @@ -622,7 +634,7 @@ public void testMatchWithExtractionFn() ), ImmutableList.of("1", "2", "5") ); - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter( "dim2", "super-null", @@ -636,7 +648,7 @@ public void testMatchWithExtractionFn() ImmutableList.of("1", "2", "5") ); } else { - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter( "dim2", "super-null", @@ -649,11 +661,11 @@ public void testMatchWithExtractionFn() ), ImmutableList.of("1", "5") ); - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "super-", "super-", false, false, false, superFn, StringComparators.NUMERIC), ImmutableList.of("2") ); - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter( "dim2", "super-null", @@ -666,7 +678,7 @@ public void testMatchWithExtractionFn() ), ImmutableList.of("1", "5") ); - assertFilterMatches( + assertFilterMatchesSkipArrays( new BoundDimFilter("dim2", "super-", "super-", false, false, false, superFn, StringComparators.NUMERIC), ImmutableList.of("2") ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java index 2bc1c2c5c4f3..8a46d313d2d0 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ColumnComparisonFilterTest.java @@ -97,7 +97,7 @@ public void testColumnsWithoutNulls() { // columns have mixed type input and so are ingested as COMPLEX // however the comparison filter currently nulls out complex types instead of comparing the values - if (testName.contains("AutoTypes")) { + if (isAutoSchema()) { return; } assertFilterMatchesSkipVectorize(new ColumnComparisonDimFilter(ImmutableList.of( @@ -124,7 +124,7 @@ public void testMissingColumnNotSpecifiedInDimensionList() { // columns have mixed type input and so are ingested as COMPLEX // however the comparison filter currently nulls out complex types instead of comparing the values - if (testName.contains("AutoTypes")) { + if (isAutoSchema()) { return; } assertFilterMatchesSkipVectorize(new ColumnComparisonDimFilter(ImmutableList.of( diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTest.java new file mode 100644 index 000000000000..7a63b3f6999f --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTest.java @@ -0,0 +1,678 @@ +/* + * 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.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.query.extraction.TimeDimExtractionFn; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.query.lookup.LookupExtractionFn; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.Closeable; +import java.util.Map; + +@RunWith(Parameterized.class) +public class EqualityFilterTest extends BaseFilterTest +{ + public EqualityFilterTest( + String testName, + IndexBuilder indexBuilder, + Function> finisher, + boolean cnf, + boolean optimize + ) + { + super(testName, DEFAULT_ROWS, indexBuilder, finisher, cnf, optimize); + } + + @AfterClass + public static void tearDown() throws Exception + { + BaseFilterTest.tearDown(EqualityFilterTest.class.getName()); + } + + @Test + public void testInvalidParameters() + { + Throwable t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new EqualityFilter(null, ColumnType.STRING, null, null, null), ImmutableList.of() + ) + ); + Assert.assertEquals("Invalid equality filter, column cannot be null", t.getMessage()); + t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new EqualityFilter("dim0", null, null, null, null), ImmutableList.of() + ) + ); + Assert.assertEquals("Invalid equality filter on column [dim0], matchValueType cannot be null", t.getMessage()); + t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new EqualityFilter("dim0", ColumnType.STRING, null, null, null), ImmutableList.of() + ) + ); + Assert.assertEquals("Invalid equality filter on column [dim0], matchValue cannot be null", t.getMessage()); + } + + @Test + public void testSingleValueStringColumnWithoutNulls() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("dim0", ColumnType.STRING, "", null, null), ImmutableList.of()); + } + assertFilterMatches(new EqualityFilter("dim0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("dim0", ColumnType.STRING, "1", null, null), ImmutableList.of("1")); + + assertFilterMatches(new EqualityFilter("dim0", ColumnType.LONG, 0L, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("dim0", ColumnType.LONG, 1L, null, null), ImmutableList.of("1")); + } + + @Test + public void testSingleValueVirtualStringColumnWithoutNulls() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("vdim0", ColumnType.STRING, "", null, null), ImmutableList.of()); + } + assertFilterMatches(new EqualityFilter("vdim0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("vdim0", ColumnType.STRING, "1", null, null), ImmutableList.of("1")); + assertFilterMatches(new EqualityFilter("vdim0", ColumnType.LONG, 0L, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("vdim0", ColumnType.LONG, 1L, null, null), ImmutableList.of("1")); + } + + @Test + public void testListFilteredVirtualColumn() + { + assertFilterMatchesSkipVectorize( + new EqualityFilter("allow-dim0", ColumnType.STRING, "1", null, null), + ImmutableList.of() + ); + assertFilterMatchesSkipVectorize( + new EqualityFilter("allow-dim0", ColumnType.STRING, "4", null, null), + ImmutableList.of("4") + ); + assertFilterMatchesSkipVectorize( + new EqualityFilter("deny-dim0", ColumnType.STRING, "0", null, null), + ImmutableList.of("0") + ); + assertFilterMatchesSkipVectorize( + new EqualityFilter("deny-dim0", ColumnType.STRING, "4", null, null), + ImmutableList.of() + ); + + // auto ingests arrays instead of MVDs which dont work with list filtered virtual column + if (!isAutoSchema()) { + assertFilterMatchesSkipVectorize( + new EqualityFilter("allow-dim2", ColumnType.STRING, "b", null, null), + ImmutableList.of() + ); + assertFilterMatchesSkipVectorize( + new EqualityFilter("allow-dim2", ColumnType.STRING, "a", null, null), + ImmutableList.of("0", "3") + ); + assertFilterMatchesSkipVectorize( + new EqualityFilter("deny-dim2", ColumnType.STRING, "b", null, null), + ImmutableList.of("0") + ); + assertFilterMatchesSkipVectorize( + new EqualityFilter("deny-dim2", ColumnType.STRING, "a", null, null), + ImmutableList.of() + ); + } + } + + @Test + public void testSingleValueStringColumnWithNulls() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "", null, null), ImmutableList.of("0")); + } + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "10", null, null), ImmutableList.of("1")); + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "2", null, null), ImmutableList.of("2")); + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "1", null, null), ImmutableList.of("3")); + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "abdef", null, null), ImmutableList.of("4")); + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "abc", null, null), ImmutableList.of("5")); + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "ab", null, null), ImmutableList.of()); + } + + @Test + public void testSingleValueVirtualStringColumnWithNulls() + { + // testSingleValueStringColumnWithNulls but with virtual column selector + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "", null, null), ImmutableList.of("0")); + } + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "10", null, null), ImmutableList.of("1")); + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "2", null, null), ImmutableList.of("2")); + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "1", null, null), ImmutableList.of("3")); + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "abdef", null, null), ImmutableList.of("4")); + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "abc", null, null), ImmutableList.of("5")); + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "ab", null, null), ImmutableList.of()); + } + + @Test + public void testMultiValueStringColumn() + { + if (isAutoSchema()) { + // auto ingests arrays instead of strings + // single values are implicitly upcast to single element arrays, so we get some matches here... + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "", null, null), ImmutableList.of("2")); + } + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "a", null, null), ImmutableList.of("3")); + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "b", null, null), ImmutableList.of()); + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "c", null, null), ImmutableList.of("4")); + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "d", null, null), ImmutableList.of()); + + // array matchers can match the whole array + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING, ImmutableList.of(""), null, null), + ImmutableList.of("2") + ); + } + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING_ARRAY, new Object[]{"a", "b"}, null, null), + ImmutableList.of("0") + ); + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING_ARRAY, ImmutableList.of("a", "b"), null, null), + ImmutableList.of("0") + ); + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING_ARRAY, new Object[]{"a"}, null, null), + ImmutableList.of("3") + ); + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING_ARRAY, new Object[]{"b"}, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING_ARRAY, new Object[]{"c"}, null, null), + ImmutableList.of("4") + ); + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING_ARRAY, new Object[]{"d"}, null, null), + ImmutableList.of() + ); + } else { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "", null, null), ImmutableList.of("2")); + } + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING, "a", null, null), + ImmutableList.of("0", "3") + ); + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "b", null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "c", null, null), ImmutableList.of("4")); + assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "d", null, null), ImmutableList.of()); + } + } + + @Test + public void testMissingColumnSpecifiedInDimensionList() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("dim3", ColumnType.STRING, "", null, null), ImmutableList.of()); + } + assertFilterMatches(new EqualityFilter("dim3", ColumnType.STRING, "a", null, null), ImmutableList.of()); + assertFilterMatches(new EqualityFilter("dim3", ColumnType.STRING, "b", null, null), ImmutableList.of()); + assertFilterMatches(new EqualityFilter("dim3", ColumnType.STRING, "c", null, null), ImmutableList.of()); + } + + @Test + public void testMissingColumnNotSpecifiedInDimensionList() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(new EqualityFilter("dim4", ColumnType.STRING, "", null, null), ImmutableList.of()); + } + assertFilterMatches(new EqualityFilter("dim4", ColumnType.STRING, "a", null, null), ImmutableList.of()); + assertFilterMatches(new EqualityFilter("dim4", ColumnType.STRING, "b", null, null), ImmutableList.of()); + assertFilterMatches(new EqualityFilter("dim4", ColumnType.STRING, "c", null, null), ImmutableList.of()); + } + + @Test + public void testExpressionVirtualColumn() + { + assertFilterMatches( + new EqualityFilter("expr", ColumnType.STRING, "1.1", null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches(new EqualityFilter("expr", ColumnType.STRING, "1.2", null, null), ImmutableList.of()); + + assertFilterMatches( + new EqualityFilter("expr", ColumnType.FLOAT, 1.1f, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches(new EqualityFilter("expr", ColumnType.FLOAT, 1.2f, null, null), ImmutableList.of()); + + assertFilterMatches( + new EqualityFilter("expr", ColumnType.DOUBLE, 1.1, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches(new EqualityFilter("expr", ColumnType.DOUBLE, 1.2, null, null), ImmutableList.of()); + + // if we accidentally specify it as a string, it works too... + assertFilterMatches( + new EqualityFilter("expr", ColumnType.DOUBLE, "1.1", null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches(new EqualityFilter("expr", ColumnType.DOUBLE, "1.2", null, null), ImmutableList.of()); + } + + @Test + public void testWithTimeExtractionFnNull() + { + assertFilterMatches(new EqualityFilter( + "timeDim", + ColumnType.STRING, + "2017-07", + new TimeDimExtractionFn("yyyy-MM-dd", "yyyy-MM", true), + null + ), ImmutableList.of("0", "1")); + assertFilterMatches(new EqualityFilter( + "timeDim", + ColumnType.STRING, + "2017-05", + new TimeDimExtractionFn("yyyy-MM-dd", "yyyy-MM", true), + null + ), ImmutableList.of("2")); + + assertFilterMatches(new EqualityFilter( + "timeDim", + ColumnType.STRING, + "2020-01", + new TimeDimExtractionFn("yyyy-MM-dd", "yyyy-MM", true), + null + ), ImmutableList.of("3", "5")); + } + + @Test + public void testSelectorWithLookupExtractionFn() + { + final Map stringMap = ImmutableMap.of( + "1", "HELLO", + "a", "HELLO", + "abdef", "HELLO", + "abc", "UNKNOWN" + ); + LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false); + LookupExtractionFn lookupFn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true); + + assertFilterMatches(new EqualityFilter("dim0", ColumnType.STRING, "HELLO", lookupFn, null), ImmutableList.of("1")); + assertFilterMatches( + new EqualityFilter("dim0", ColumnType.STRING, "UNKNOWN", lookupFn, null), + ImmutableList.of("0", "2", "3", "4", "5") + ); + + assertFilterMatches( + new EqualityFilter("dim1", ColumnType.STRING, "HELLO", lookupFn, null), + ImmutableList.of("3", "4") + ); + assertFilterMatches( + new EqualityFilter("dim1", ColumnType.STRING, "UNKNOWN", lookupFn, null), + ImmutableList.of("0", "1", "2", "5") + ); + + if (!isAutoSchema()) { + // auto ingests arrays instead of mvds, arrays do not support extractionFn + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING, "HELLO", lookupFn, null), + ImmutableList.of("0", "3") + ); + assertFilterMatches( + new EqualityFilter("dim2", ColumnType.STRING, "UNKNOWN", lookupFn, null), + ImmutableList.of("0", "1", "2", "4", "5") + ); + + assertFilterMatches( + new EqualityFilter("dim3", ColumnType.STRING, "HELLO", lookupFn, null), + ImmutableList.of() + ); + assertFilterMatches( + new EqualityFilter("dim3", ColumnType.STRING, "UNKNOWN", lookupFn, null), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + assertFilterMatches( + new EqualityFilter("dim4", ColumnType.STRING, "HELLO", lookupFn, null), + ImmutableList.of() + ); + assertFilterMatches( + new EqualityFilter("dim4", ColumnType.STRING, "UNKNOWN", lookupFn, null), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + } + + final Map stringMap2 = ImmutableMap.of( + "2", "5" + ); + LookupExtractor mapExtractor2 = new MapLookupExtractor(stringMap2, false); + LookupExtractionFn lookupFn2 = new LookupExtractionFn(mapExtractor2, true, null, false, true); + assertFilterMatches( + new EqualityFilter("dim0", ColumnType.STRING, "5", lookupFn2, null), + ImmutableList.of("2", "5") + ); + + if (NullHandling.sqlCompatible()) { + + final Map stringMap3 = ImmutableMap.of( + "1", "" + ); + LookupExtractor mapExtractor3 = new MapLookupExtractor(stringMap3, false); + LookupExtractionFn lookupFn3 = new LookupExtractionFn(mapExtractor3, false, null, false, true); + assertFilterMatches( + new EqualityFilter("dim0", ColumnType.STRING, "", lookupFn3, null), + ImmutableList.of("1") + ); + } + } + + @Test + public void testNumericColumnNullsAndDefaults() + { + if (canTestNumericNullsAsDefaultValues) { + assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 0f, null, null), ImmutableList.of("0", "4")); + assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 0.0, null, null), ImmutableList.of("0", "2")); + assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 0L, null, null), ImmutableList.of("0", "3")); + assertFilterMatches(new EqualityFilter("f0", ColumnType.STRING, "0", null, null), ImmutableList.of("0", "4")); + assertFilterMatches(new EqualityFilter("d0", ColumnType.STRING, "0", null, null), ImmutableList.of("0", "2")); + assertFilterMatches(new EqualityFilter("l0", ColumnType.STRING, "0", null, null), ImmutableList.of("0", "3")); + } else { + assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 0f, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 0.0, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 0L, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("f0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("d0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("l0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + } + } + + @Test + public void testVirtualNumericColumnNullsAndDefaults() + { + if (canTestNumericNullsAsDefaultValues) { + assertFilterMatches(new EqualityFilter("vf0", ColumnType.FLOAT, 0f, null, null), ImmutableList.of("0", "4")); + assertFilterMatches(new EqualityFilter("vd0", ColumnType.DOUBLE, 0.0, null, null), ImmutableList.of("0", "2")); + assertFilterMatches(new EqualityFilter("vl0", ColumnType.LONG, 0L, null, null), ImmutableList.of("0", "3")); + assertFilterMatches(new EqualityFilter("vf0", ColumnType.STRING, "0", null, null), ImmutableList.of("0", "4")); + assertFilterMatches(new EqualityFilter("vd0", ColumnType.STRING, "0", null, null), ImmutableList.of("0", "2")); + assertFilterMatches(new EqualityFilter("vl0", ColumnType.STRING, "0", null, null), ImmutableList.of("0", "3")); + } else { + assertFilterMatches(new EqualityFilter("vf0", ColumnType.FLOAT, 0f, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("vd0", ColumnType.DOUBLE, 0.0, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("vl0", ColumnType.LONG, 0L, null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("vf0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("vd0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + assertFilterMatches(new EqualityFilter("vl0", ColumnType.STRING, "0", null, null), ImmutableList.of("0")); + } + } + + @Test + public void testNumeric() + { + /* + dim0 d0 f0 l0 + "0" .. 0.0, 0.0f, 0L + "1" .. 10.1, 10.1f, 100L + "2" .. null, 5.5f, 40L + "3" .. 120.0245, 110.0f, null + "4" .. 60.0, null, 9001L + "5" .. 765.432, 123.45f, 12345L + */ + + assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 10.1, null, null), ImmutableList.of("1")); + assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 120.0245, null, null), ImmutableList.of("3")); + assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 765.432, null, null), ImmutableList.of("5")); + assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 765.431, null, null), ImmutableList.of()); + + assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 100L, null, null), ImmutableList.of("1")); + assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 40L, null, null), ImmutableList.of("2")); + assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 9001L, null, null), ImmutableList.of("4")); + assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 9000L, null, null), ImmutableList.of()); + if (!isAutoSchema()) { + // auto schema doesn't store float columns as floats, rather they are stored as doubles... the predicate matcher + // matches fine, but the string value set index does not match correctly if we expect the input float values + assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 10.1f, null, null), ImmutableList.of("1")); + assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 110.0f, null, null), ImmutableList.of("3")); + assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 123.45f, null, null), ImmutableList.of("5")); + assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 123.46f, null, null), ImmutableList.of()); + } else { + // .. so we need to cast them instead + assertFilterMatches( + new EqualityFilter("f0", ColumnType.DOUBLE, (double) 10.1f, null, null), + ImmutableList.of("1") + ); + assertFilterMatches( + new EqualityFilter("f0", ColumnType.DOUBLE, (double) 110.0f, null, null), + ImmutableList.of("3") + ); + assertFilterMatches( + new EqualityFilter("f0", ColumnType.DOUBLE, (double) 123.45f, null, null), + ImmutableList.of("5") + ); + assertFilterMatches( + new EqualityFilter("f0", ColumnType.DOUBLE, (double) 123.46f, null, null), + ImmutableList.of() + ); + } + } + + @Test + public void testArrays() + { + if (isAutoSchema()) { + // only auto schema supports array columns... skip other segment types + /* + dim0 .. arrayString arrayLong arrayDouble + "0", .. ["a", "b", "c"], [1L, 2L, 3L], [1.1, 2.2, 3.3] + "1", .. [], [], [1.1, 2.2, 3.3] + "2", .. null, [1L, 2L, 3L], [null] + "3", .. ["a", "b", "c"], null, [] + "4", .. ["c", "d"], [null], [-1.1, -333.3] + "5", .. [null], [123L, 345L], null + */ + + assertFilterMatches( + new EqualityFilter( + "arrayString", + ColumnType.STRING_ARRAY, + ImmutableList.of("a", "b", "c"), + null, + null + ), + ImmutableList.of("0", "3") + ); + assertFilterMatches( + new EqualityFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{"a", "b", "c"}, + null, + null + ), + ImmutableList.of("0", "3") + ); + assertFilterMatches( + new EqualityFilter( + "arrayString", + ColumnType.STRING_ARRAY, + ImmutableList.of(), + null, + null + ), + ImmutableList.of("1") + ); + assertFilterMatches( + new EqualityFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{null}, + null, + null + ), + ImmutableList.of("5") + ); + assertFilterMatches( + new EqualityFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{null, null}, + null, + null + ), + ImmutableList.of() + ); + assertFilterMatches( + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + ImmutableList.of(1L, 2L, 3L), + null, + null + ), + ImmutableList.of("0", "2") + ); + assertFilterMatches( + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{1L, 2L, 3L}, + null, + null + ), + ImmutableList.of("0", "2") + ); + assertFilterMatches( + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + ImmutableList.of(), + null, + null + ), + ImmutableList.of("1") + ); + assertFilterMatches( + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{null}, + null, + null + ), + ImmutableList.of("4") + ); + assertFilterMatches( + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{null, null}, + null, + null + ), + ImmutableList.of() + ); + assertFilterMatches( + new EqualityFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + ImmutableList.of(1.1, 2.2, 3.3), + null, + null + ), + ImmutableList.of("0", "1") + ); + assertFilterMatches( + new EqualityFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + new Object[]{1.1, 2.2, 3.3}, + null, + null + ), + ImmutableList.of("0", "1") + ); + assertFilterMatches( + new EqualityFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + ImmutableList.of(), + null, + null + ), + ImmutableList.of("3") + ); + assertFilterMatches( + new EqualityFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + new Object[]{null}, + null, + null + ), + ImmutableList.of("2") + ); + assertFilterMatches( + new EqualityFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + ImmutableList.of(1.1, 2.2, 3.4), + null, + null + ), + ImmutableList.of() + ); + } + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(EqualityFilter.class).usingGetClass() + .withNonnullFields( + "column", + "matchValueType", + "matchValue", + "predicateFactory", + "cachedOptimizedFilter" + ) + .withPrefabValues(ColumnType.class, ColumnType.STRING, ColumnType.DOUBLE) + .withIgnoredFields("predicateFactory", "cachedOptimizedFilter") + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java index e46e736f9fa9..ac0ef94c2000 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java @@ -140,6 +140,10 @@ public static void tearDown() throws Exception @Test public void testOneSingleValuedStringColumn() { + if (testName.contains("incrementalAutoTypes")) { + // dim 3 is mixed type in auto incrementalIndex, so presents as complex + return; + } assertFilterMatches(edf("dim3 == ''"), ImmutableList.of("0")); assertFilterMatches(edf("dim3 == '1'"), ImmutableList.of("3", "4", "6")); assertFilterMatches(edf("dim3 == 'a'"), ImmutableList.of("7")); @@ -162,8 +166,8 @@ public void testOneSingleValuedStringColumn() @Test public void testOneMultiValuedStringColumn() { - // auto type columns don't support mvds, bail out - if (testName.contains("AutoTypes")) { + // auto type columns ingest arrays instead of mvds, bail out + if (isAutoSchema()) { return; } if (NullHandling.replaceWithDefault()) { @@ -252,8 +256,8 @@ public void testCompareColumns() assertFilterMatches(edf("dim2 == dim3"), ImmutableList.of("2", "5", "8")); } - // auto type columns don't support mvds, bail out - if (testName.contains("AutoTypes")) { + // auto type columns ingest arrays instead of mvds + if (isAutoSchema()) { return; } // String vs. multi-value string diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index 0c6733e648be..3e19b9a113df 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -44,8 +44,8 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Test; @@ -235,6 +235,9 @@ public void testSinglePostFilterWithNulls() @Test public void testBasicPreAndPostFilterWithNulls() { + if (isAutoSchema()) { + return; + } if (NullHandling.replaceWithDefault()) { assertFilterMatches( new AndDimFilter(Arrays.asList( @@ -359,6 +362,9 @@ public void testBasicPreAndPostFilterWithNulls() @Test public void testOrPostFilterWithNulls() { + if (isAutoSchema()) { + return; + } assertFilterMatches( new OrDimFilter(Arrays.asList( new SelectorDimFilter("dim2", "a", null), @@ -617,13 +623,17 @@ public void testMissingColumnNotSpecifiedInDimensionList() @Test public void testDistributeOrCNF() throws CNFFilterExplosionException { + if (isAutoSchema()) { + return; + } DimFilter dimFilter1 = new OrDimFilter(Arrays.asList( new SelectorDimFilter("dim0", "6", null), new AndDimFilter(Arrays.asList( new NoBitmapSelectorDimFilter("dim1", "abdef", null), new SelectorDimFilter("dim2", "c", null) ) - )) + ) + ) ); Filter filter1 = dimFilter1.toFilter(); @@ -671,13 +681,17 @@ public void testDistributeOrCNF() throws CNFFilterExplosionException @Test public void testDistributeOrCNFExtractionFn() throws CNFFilterExplosionException { + if (isAutoSchema()) { + return; + } DimFilter dimFilter1 = new OrDimFilter(Arrays.asList( new SelectorDimFilter("dim0", "super-6", JS_EXTRACTION_FN), new AndDimFilter(Arrays.asList( new NoBitmapSelectorDimFilter("dim1", "super-abdef", JS_EXTRACTION_FN), new SelectorDimFilter("dim2", "super-c", JS_EXTRACTION_FN) ) - )) + ) + ) ); Filter filter1 = dimFilter1.toFilter(); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java index 23ced8e5208a..535a134b0354 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java @@ -149,6 +149,9 @@ public void testSingleValueStringColumnWithNulls() @Test public void testMultiValueStringColumn() { + if (isAutoSchema()) { + return; + } if (NullHandling.replaceWithDefault()) { assertFilterMatches( toInFilter("dim2", "b", "d"), @@ -256,35 +259,35 @@ public void testMatchWithExtractionFn() ExtractionFn yesNullFn = new JavaScriptExtractionFn(nullJsFn, false, JavaScriptConfig.getEnabledInstance()); if (NullHandling.replaceWithDefault()) { - assertFilterMatches( + assertFilterMatchesSkipArrays( toInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b"), ImmutableList.of("a", "b", "c", "d", "f") ); + assertFilterMatchesSkipArrays( + toInFilterWithFn("dim2", yesNullFn, "YES"), + ImmutableList.of("b", "c", "f") + ); assertFilterMatches( toInFilterWithFn("dim1", superFn, "super-null", "super-10", "super-def"), ImmutableList.of("a", "b", "e") ); - assertFilterMatches( - toInFilterWithFn("dim2", yesNullFn, "YES"), - ImmutableList.of("b", "c", "f") - ); assertFilterMatches( toInFilterWithFn("dim1", yesNullFn, "NO"), ImmutableList.of("b", "c", "d", "e", "f") ); } else { - assertFilterMatches( + assertFilterMatchesSkipArrays( toInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b"), ImmutableList.of("a", "b", "d", "f") ); + assertFilterMatchesSkipArrays( + toInFilterWithFn("dim2", yesNullFn, "YES"), + ImmutableList.of("b", "f") + ); assertFilterMatches( toInFilterWithFn("dim1", superFn, "super-null", "super-10", "super-def"), ImmutableList.of("b", "e") ); - assertFilterMatches( - toInFilterWithFn("dim2", yesNullFn, "YES"), - ImmutableList.of("b", "f") - ); assertFilterMatches( toInFilterWithFn("dim1", yesNullFn, "NO"), @@ -322,9 +325,10 @@ public void testMatchWithLookupExtractionFn() assertFilterMatches(toInFilterWithFn("dim0", lookupFn, "UNKNOWN"), ImmutableList.of("b", "d", "e", "f")); assertFilterMatches(toInFilterWithFn("dim1", lookupFn, "HELLO"), ImmutableList.of("b", "e")); assertFilterMatches(toInFilterWithFn("dim1", lookupFn, "N/A"), ImmutableList.of()); - assertFilterMatches(toInFilterWithFn("dim2", lookupFn, "a"), ImmutableList.of()); - assertFilterMatches(toInFilterWithFn("dim2", lookupFn, "HELLO"), ImmutableList.of("a", "d")); - assertFilterMatches( + + assertFilterMatchesSkipArrays(toInFilterWithFn("dim2", lookupFn, "a"), ImmutableList.of()); + assertFilterMatchesSkipArrays(toInFilterWithFn("dim2", lookupFn, "HELLO"), ImmutableList.of("a", "d")); + assertFilterMatchesSkipArrays( toInFilterWithFn("dim2", lookupFn, "HELLO", "BYE", "UNKNOWN"), ImmutableList.of("a", "b", "c", "d", "e", "f") ); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java index 40ccaa64dbb4..98b9579a9079 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/JavaScriptFilterTest.java @@ -111,6 +111,9 @@ public void testSingleValueStringColumnWithNulls() @Test public void testMultiValueStringColumn() { + if (isAutoSchema()) { + return; + } // multi-val null...... if (NullHandling.replaceWithDefault()) { assertFilterMatchesSkipVectorize( @@ -184,14 +187,16 @@ public void testJavascriptFilterWithLookupExtractionFn() ImmutableList.of("0", "1", "2", "5") ); - assertFilterMatchesSkipVectorize( - newJavaScriptDimFilter("dim2", jsValueFilter("HELLO"), lookupFn), - ImmutableList.of("0", "3") - ); - assertFilterMatchesSkipVectorize( - newJavaScriptDimFilter("dim2", jsValueFilter("UNKNOWN"), lookupFn), - ImmutableList.of("0", "1", "2", "4", "5") - ); + if (!isAutoSchema()) { + assertFilterMatchesSkipVectorize( + newJavaScriptDimFilter("dim2", jsValueFilter("HELLO"), lookupFn), + ImmutableList.of("0", "3") + ); + assertFilterMatchesSkipVectorize( + newJavaScriptDimFilter("dim2", jsValueFilter("UNKNOWN"), lookupFn), + ImmutableList.of("0", "1", "2", "4", "5") + ); + } assertFilterMatchesSkipVectorize( newJavaScriptDimFilter("dim3", jsValueFilter("HELLO"), lookupFn), diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTest.java new file mode 100644 index 000000000000..eebba4c75f24 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTest.java @@ -0,0 +1,269 @@ +/* + * 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.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.extraction.MapLookupExtractor; +import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.query.lookup.LookupExtractionFn; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.StorageAdapter; +import org.junit.AfterClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.Closeable; +import java.util.Map; + +@RunWith(Parameterized.class) +public class NullFilterTest extends BaseFilterTest +{ + public NullFilterTest( + String testName, + IndexBuilder indexBuilder, + Function> finisher, + boolean cnf, + boolean optimize + ) + { + super(testName, DEFAULT_ROWS, indexBuilder, finisher, cnf, optimize); + } + + @AfterClass + public static void tearDown() throws Exception + { + BaseFilterTest.tearDown(NullFilterTest.class.getName()); + } + + @Test + public void testSingleValueStringColumnWithoutNulls() + { + assertFilterMatches(NullFilter.forColumn("dim0"), ImmutableList.of()); + } + + @Test + public void testSingleValueVirtualStringColumnWithoutNulls() + { + assertFilterMatches(NullFilter.forColumn("vdim0"), ImmutableList.of()); + } + + @Test + public void testListFilteredVirtualColumn() + { + assertFilterMatchesSkipVectorize(NullFilter.forColumn("allow-dim0"), ImmutableList.of("0", "1", "2", "5")); + assertFilterMatchesSkipVectorize(NullFilter.forColumn("deny-dim0"), ImmutableList.of("3", "4")); + if (isAutoSchema()) { + return; + } + assertFilterMatchesSkipVectorize(NullFilter.forColumn("allow-dim2"), ImmutableList.of("1", "2", "4", "5")); + if (NullHandling.replaceWithDefault()) { + assertFilterMatchesSkipVectorize( + NullFilter.forColumn("deny-dim2"), + ImmutableList.of("1", "2", "3", "5") + ); + } else { + assertFilterMatchesSkipVectorize( + NullFilter.forColumn("deny-dim2"), + ImmutableList.of("1", "3", "5") + ); + } + } + + @Test + public void testSingleValueStringColumnWithNulls() + { + // testSingleValueStringColumnWithoutNulls but with virtual column selector + if (NullHandling.replaceWithDefault()) { + assertFilterMatches(NullFilter.forColumn("dim1"), ImmutableList.of("0")); + } else { + assertFilterMatches(NullFilter.forColumn("dim1"), ImmutableList.of()); + } + } + + @Test + public void testSingleValueVirtualStringColumnWithNulls() + { + // testSingleValueStringColumnWithNulls but with virtual column selector + if (NullHandling.replaceWithDefault()) { + assertFilterMatches(NullFilter.forColumn("vdim1"), ImmutableList.of("0")); + } else { + assertFilterMatches(NullFilter.forColumn("vdim1"), ImmutableList.of()); + } + } + + @Test + public void testMultiValueStringColumn() + { + if (NullHandling.replaceWithDefault()) { + if (isAutoSchema()) { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("5")); + } else { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("1", "2", "5")); + } + } else { + // only one array row is totally null + if (isAutoSchema()) { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("5")); + } else { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("1", "5")); + } + } + } + + @Test + public void testMissingColumnSpecifiedInDimensionList() + { + assertFilterMatches(NullFilter.forColumn("dim3"), ImmutableList.of("0", "1", "2", "3", "4", "5")); + } + + @Test + public void testMissingColumnNotSpecifiedInDimensionList() + { + assertFilterMatches(NullFilter.forColumn("dim4"), ImmutableList.of("0", "1", "2", "3", "4", "5")); + } + + + @Test + public void testVirtualNumericColumnNullsAndDefaults() + { + if (canTestNumericNullsAsDefaultValues) { + assertFilterMatches(NullFilter.forColumn("vf0"), ImmutableList.of()); + assertFilterMatches(NullFilter.forColumn("vd0"), ImmutableList.of()); + assertFilterMatches(NullFilter.forColumn("vl0"), ImmutableList.of()); + } else { + assertFilterMatches(NullFilter.forColumn("vf0"), ImmutableList.of("4")); + assertFilterMatches(NullFilter.forColumn("vd0"), ImmutableList.of("2")); + assertFilterMatches(NullFilter.forColumn("vl0"), ImmutableList.of("3")); + } + } + + @Test + public void testNumericColumnNullsAndDefaults() + { + if (canTestNumericNullsAsDefaultValues) { + assertFilterMatches(NullFilter.forColumn("f0"), ImmutableList.of()); + assertFilterMatches(NullFilter.forColumn("d0"), ImmutableList.of()); + assertFilterMatches(NullFilter.forColumn("l0"), ImmutableList.of()); + } else { + assertFilterMatches(NullFilter.forColumn("f0"), ImmutableList.of("4")); + assertFilterMatches(NullFilter.forColumn("d0"), ImmutableList.of("2")); + assertFilterMatches(NullFilter.forColumn("l0"), ImmutableList.of("3")); + } + } + + @Test + public void testSelectorWithLookupExtractionFn() + { + /* + static final List DEFAULT_ROWS = ImmutableList.of( + makeDefaultSchemaRow("0", "", ImmutableList.of("a", "b"), "2017-07-25", 0.0, 0.0f, 0L), + makeDefaultSchemaRow("1", "10", ImmutableList.of(), "2017-07-25", 10.1, 10.1f, 100L), + makeDefaultSchemaRow("2", "2", ImmutableList.of(""), "2017-05-25", null, 5.5f, 40L), + makeDefaultSchemaRow("3", "1", ImmutableList.of("a"), "2020-01-25", 120.0245, 110.0f, null), + makeDefaultSchemaRow("4", "abdef", ImmutableList.of("c"), null, 60.0, null, 9001L), + makeDefaultSchemaRow("5", "abc", null, "2020-01-25", 765.432, 123.45f, 12345L) + ); + */ + final Map stringMap = ImmutableMap.of( + "1", "HELLO", + "a", "HELLO", + "abdef", "HELLO", + "abc", "UNKNOWN" + ); + LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false); + LookupExtractionFn lookupFn = new LookupExtractionFn(mapExtractor, false, null, false, true); + LookupExtractionFn lookupFnRetain = new LookupExtractionFn(mapExtractor, true, null, false, true); + LookupExtractionFn lookupFnReplace = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true); + + if (NullHandling.replaceWithDefault()) { + assertFilterMatches(new NullFilter("dim0", lookupFn, null), ImmutableList.of("0", "2", "3", "4", "5")); + assertFilterMatches(new NullFilter("dim0", lookupFnRetain, null), ImmutableList.of()); + } else { + assertFilterMatches(new NullFilter("dim0", lookupFn, null), ImmutableList.of("0", "2", "3", "4", "5")); + assertFilterMatches(new NullFilter("dim0", lookupFnRetain, null), ImmutableList.of()); + } + + assertFilterMatches(new NullFilter("dim0", lookupFnReplace, null), ImmutableList.of()); + + + final Map stringMapEmpty = ImmutableMap.of( + "1", "" + ); + LookupExtractor mapExtractoryEmpty = new MapLookupExtractor(stringMapEmpty, false); + LookupExtractionFn lookupFnEmpty = new LookupExtractionFn(mapExtractoryEmpty, false, null, false, true); + if (NullHandling.replaceWithDefault()) { + // Nulls and empty strings are considered equivalent + assertFilterMatches( + new NullFilter("dim0", lookupFnEmpty, null), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + } else { + assertFilterMatches( + new NullFilter("dim0", lookupFnEmpty, null), + ImmutableList.of("0", "2", "3", "4", "5") + ); + } + } + + @Test + public void testArrays() + { + if (isAutoSchema()) { + // only auto schema ingests arrays + /* + dim0 .. arrayString arrayLong arrayDouble + "0", .. ["a", "b", "c"], [1L, 2L, 3L], [1.1, 2.2, 3.3] + "1", .. [], [], [1.1, 2.2, 3.3] + "2", .. null, [1L, 2L, 3L], [null] + "3", .. ["a", "b", "c"], null, [] + "4", .. ["c", "d"], [null], [-1.1, -333.3] + "5", .. [null], [123L, 345L], null + */ + assertFilterMatches( + new NullFilter("arrayString", null, null), + ImmutableList.of("2") + ); + assertFilterMatches( + new NullFilter("arrayLong", null, null), + ImmutableList.of("3") + ); + assertFilterMatches( + new NullFilter("arrayDouble", null, null), + ImmutableList.of("5") + ); + } + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(NullFilter.class).usingGetClass() + .withNonnullFields("column") + .withIgnoredFields("cachedOptimizedFilter") + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTest.java new file mode 100644 index 000000000000..622f2ea8c95a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTest.java @@ -0,0 +1,1338 @@ +/* + * 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.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.InputRow; +import org.apache.druid.error.DruidException; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.js.JavaScriptConfig; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.extraction.JavaScriptExtractionFn; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.RangeFilter; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.Closeable; +import java.util.List; + +@RunWith(Parameterized.class) +public class RangeFilterTest extends BaseFilterTest +{ + private static final List ROWS = + ImmutableList.builder() + .addAll(DEFAULT_ROWS) + .add(makeDefaultSchemaRow( + "6", + "-1000", + ImmutableList.of("a"), + null, + 6.6, + null, + 10L, + new Object[]{"x", "y"}, + new Object[]{100, 200}, + new Object[]{1.1, null, 3.3} + )) + .add(makeDefaultSchemaRow( + "7", + "-10.012", + ImmutableList.of("d"), + null, + null, + 3.0f, + null, + new Object[]{null, "hello", "world"}, + new Object[]{1234, 3456L, null}, + new Object[]{1.23, 4.56, 6.78} + )) + .build(); + + public RangeFilterTest( + String testName, + IndexBuilder indexBuilder, + Function> finisher, + boolean cnf, + boolean optimize + ) + { + super(testName, ROWS, indexBuilder, finisher, cnf, optimize); + } + + @AfterClass + public static void tearDown() throws Exception + { + BaseFilterTest.tearDown(RangeFilterTest.class.getName()); + } + + @Test + public void testLexicographicalMatch() + { + + assertFilterMatches( + new RangeFilter("dim0", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + assertFilterMatches( + new RangeFilter("vdim0", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + assertFilterMatches( + new RangeFilter("vdim0", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + } else { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of("1", "2", "3", "4", "5", "6", "7") + ); + assertFilterMatches( + new RangeFilter("vdim0", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + } + + if (isAutoSchema()) { + // auto schema ingests arrays instead of mvds.. this filter doesn't currently support arrays + } else { + assertFilterMatches( + new RangeFilter("dim2", ColumnType.STRING, null, "z", false, false, null, null), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "2", "3", "4", "6", "7") + : ImmutableList.of("0", "3", "4", "6", "7") + ); + // vdim2 does not exist... + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of() + ); + } + } + + @Test + public void testLexicographicMatchWithEmptyString() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + new RangeFilter("dim0", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + if (!isAutoSchema()) { + // auto schema ingests arrays which are currently incompatible with the range filter + assertFilterMatches( + new RangeFilter("dim2", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of("0", "2", "3", "4", "6", "7") + ); + } + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of() + ); + } else { + assertFilterMatches( + new RangeFilter("dim0", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of("1", "2", "3", "4", "5", "6", "7") + ); + if (!isAutoSchema()) { + // auto schema ingests arrays which are currently incompatible with the range filter + assertFilterMatches( + new RangeFilter("dim2", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of("0", "3", "4", "6", "7") + ); + } + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of() + ); + } + } + + @Test + public void testLexicographicMatchNull() + { + + if (NullHandling.replaceWithDefault()) { + // in default value mode this is null on both ends... + Throwable t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new RangeFilter("dim0", ColumnType.STRING, "", "", false, false, null, null), + ImmutableList.of() + ) + ); + Assert.assertEquals( + "Invalid range filter on column [dim0], lower and upper cannot be null at the same time", + t.getMessage() + ); + } else { + assertFilterMatches( + new RangeFilter("dim0", ColumnType.STRING, "", "", false, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "", "", false, false, null, null), + ImmutableList.of("0") + ); + // still matches even with auto-schema because match-values are upcast to array types + assertFilterMatches( + new RangeFilter("dim2", ColumnType.STRING, "", "", false, false, null, null), + ImmutableList.of("2") + ); + } + } + + @Test + public void testLexicographicMatchMissingColumn() + { + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, "", "z", false, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, "a", null, false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, null, "z", false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, "", "z", true, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, "", "z", false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim3", ColumnType.STRING, null, "z", false, true, null, null), + ImmutableList.of() + ); + } + + + @Test + public void testLexicographicMatchTooStrict() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", true, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", true, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", false, true, null, null), + ImmutableList.of() + ); + } + + @Test + public void testLexicographicMatchExactlySingleValue() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", false, false, null, null), + ImmutableList.of("5") + ); + } + + @Test + public void testLexicographicMatchSurroundingSingleValue() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "ab", "abd", true, true, null, null), + ImmutableList.of("5") + ); + } + + @Test + public void testLexicographicMatchNoUpperLimit() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "ab", null, true, true, null, null), + ImmutableList.of("4", "5") + ); + } + + @Test + public void testLexicographicMatchNoLowerLimit() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, null, "abd", true, true, null, null), + NullHandling.replaceWithDefault() + ? ImmutableList.of("1", "2", "3", "5", "6", "7") + : ImmutableList.of("0", "1", "2", "3", "5", "6", "7") + ); + } + + @Test + public void testLexicographicMatchNumbers() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "1", "3", false, false, null, null), + ImmutableList.of("1", "2", "3") + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "1", "3", true, true, null, null), + ImmutableList.of("1", "2") + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "-1", "3", true, true, null, null), + ImmutableList.of("1", "2", "3", "6", "7") + ); + } + + + @Test + public void testNumericMatchBadParameters() + { + Throwable t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new RangeFilter(null, ColumnType.DOUBLE, "1234", "", false, false, null, null), + ImmutableList.of() + ) + ); + Assert.assertEquals( + "Invalid range filter, column cannot be null", + t.getMessage() + ); + t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new RangeFilter("dim0", null, "1234", "", false, false, null, null), + ImmutableList.of() + ) + ); + Assert.assertEquals( + "Invalid range filter on column [dim0], matchValueType cannot be null", + t.getMessage() + ); + t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new RangeFilter("dim0", ColumnType.DOUBLE, null, null, false, false, null, null), + ImmutableList.of() + ) + ); + Assert.assertEquals( + "Invalid range filter on column [dim0], lower and upper cannot be null at the same time", + t.getMessage() + ); + + t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new RangeFilter("dim0", ColumnType.DOUBLE, "1234", "", false, false, null, null), + ImmutableList.of() + ) + ); + Assert.assertEquals( + "Invalid range filter on column [dim0], upper bound [] cannot be parsed as specified match value type [DOUBLE]", + t.getMessage() + ); + + t = Assert.assertThrows( + DruidException.class, + () -> assertFilterMatches( + new RangeFilter("dim0", ColumnType.DOUBLE, "abc", "1234", false, false, null, null), + ImmutableList.of() + ) + ); + Assert.assertEquals( + "Invalid range filter on column [dim0], lower bound [abc] cannot be parsed as specified match value type [DOUBLE]", + t.getMessage() + ); + } + + @Test + public void testNumericMatchTooStrict() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, 2L, 2L, true, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, 2L, 2L, true, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, 2L, 2L, false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.LONG, 2L, 3L, false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.DOUBLE, 2L, 3L, false, true, null, null), + ImmutableList.of() + ); + + assertFilterMatches( + new RangeFilter("f0", ColumnType.LONG, 2L, 3L, false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("f0", ColumnType.DOUBLE, 2L, 3L, false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.LONG, 2L, 3L, false, true, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.DOUBLE, 2L, 3L, false, true, null, null), + ImmutableList.of() + ); + } + + @Test + public void testNumericMatchVirtualColumn() + { + assertFilterMatches( + new RangeFilter("expr", ColumnType.LONG, 1L, 2L, false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + assertFilterMatches( + new RangeFilter("expr", ColumnType.DOUBLE, 1.1, 2.0, false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + assertFilterMatches( + new RangeFilter("expr", ColumnType.FLOAT, 1.1f, 2.0f, false, false, null, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + + assertFilterMatches( + new RangeFilter("expr", ColumnType.LONG, 2L, 3L, false, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("expr", ColumnType.DOUBLE, 2.0, 3.0, false, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("expr", ColumnType.FLOAT, 2.0f, 3.0f, false, false, null, null), + ImmutableList.of() + ); + } + + @Test + public void testNumericMatchExactlySingleValue() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, 2L, 2L, false, false, null, null), + ImmutableList.of("2") + ); + + assertFilterMatches( + new RangeFilter("dim1", ColumnType.DOUBLE, -10.012, -10.012, false, false, null, null), + ImmutableList.of("7") + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.DOUBLE, 120.0245, 120.0245, false, false, null, null), + ImmutableList.of("3") + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.FLOAT, 120.0245f, 120.0245f, false, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.FLOAT, 60.0f, 60.0f, false, false, null, null), + ImmutableList.of("4") + ); + assertFilterMatches( + new RangeFilter("f0", ColumnType.DOUBLE, 10.1, 10.1, false, false, null, null), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("f0", ColumnType.FLOAT, 10.1f, 10.1f, false, false, null, null), + ImmutableList.of("1") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.LONG, 12345L, 12345L, false, false, null, null), + ImmutableList.of("5") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.DOUBLE, 12345.0, 12345.0, false, false, null, null), + ImmutableList.of("5") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.FLOAT, 12345.0f, 12345.0f, false, false, null, null), + ImmutableList.of("5") + ); + } + + @Test + public void testNumericMatchSurroundingSingleValue() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, 1L, 3L, true, true, null, null), + ImmutableList.of("2") + ); + + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, -11L, -10L, false, false, null, null), + ImmutableList.of("7") + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.DOUBLE, 120.0, 120.03, false, false, null, null), + ImmutableList.of("3") + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.FLOAT, 120.02f, 120.03f, false, false, null, null), + ImmutableList.of("3") + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.FLOAT, 59.5f, 60.01f, false, false, null, null), + ImmutableList.of("4") + ); + assertFilterMatches( + new RangeFilter("f0", ColumnType.DOUBLE, 10.0, 10.2, false, false, null, null), + ImmutableList.of("1") + ); + assertFilterMatches( + new RangeFilter("f0", ColumnType.FLOAT, 10.05f, 10.11f, false, false, null, null), + ImmutableList.of("1") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.LONG, 12344L, 12346L, false, false, null, null), + ImmutableList.of("5") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.DOUBLE, 12344.0, 12345.5, false, false, null, null), + ImmutableList.of("5") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.FLOAT, 12344.0f, 12345.5f, false, false, null, null), + ImmutableList.of("5") + ); + } + + @Test + public void testNumericMatchNoUpperLimit() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, 1L, null, true, true, null, null), + ImmutableList.of("1", "2") + ); + assertFilterMatches( + new RangeFilter("d0", ColumnType.DOUBLE, 1.0, null, true, true, null, null), + ImmutableList.of("1", "3", "4", "5", "6") + ); + assertFilterMatches( + new RangeFilter("f0", ColumnType.FLOAT, 1.0f, null, true, true, null, null), + ImmutableList.of("1", "2", "3", "5", "7") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.LONG, 1L, null, true, true, null, null), + ImmutableList.of("1", "2", "4", "5", "6") + ); + } + + @Test + public void testNumericMatchNoLowerLimit() + { + // strings are wierd... + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, null, 2L, false, true, null, null), + NullHandling.replaceWithDefault() + ? ImmutableList.of("3", "4", "5", "6", "7") + : ImmutableList.of("0", "3", "4", "5", "6", "7") + ); + // numbers are sane though + assertFilterMatches( + new RangeFilter("d0", ColumnType.DOUBLE, null, 10.0, false, true, null, null), + canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "2", "6", "7") : ImmutableList.of("0", "6") + ); + assertFilterMatches( + new RangeFilter("f0", ColumnType.FLOAT, null, 50.5, false, true, null, null), + canTestNumericNullsAsDefaultValues + ? ImmutableList.of("0", "1", "2", "4", "6", "7") + : ImmutableList.of("0", "1", "2", "7") + ); + assertFilterMatches( + new RangeFilter("l0", ColumnType.LONG, null, 100L, false, true, null, null), + canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "2", "3", "6", "7") : ImmutableList.of("0", "2", "6") + ); + } + + @Test + public void testNumericMatchWithNegatives() + { + assertFilterMatches( + new RangeFilter("dim1", ColumnType.LONG, -2000L, 3L, true, true, null, null), + ImmutableList.of("2", "3", "6", "7") + ); + } + + @Test + public void testNumericNullsAndZeros() + { + assertFilterMatches( + new RangeFilter( + "d0", + ColumnType.DOUBLE, + 0.0, + 1.1, + false, + false, + null, + null + ), + canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "2", "7") : ImmutableList.of("0") + ); + + assertFilterMatches( + new RangeFilter( + "f0", + ColumnType.FLOAT, + 0.0, + 1.0, + false, + false, + null, + null + ), + canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "4", "6") : ImmutableList.of("0") + ); + + assertFilterMatches( + new RangeFilter( + "l0", + ColumnType.LONG, + 0L, + 1L, + false, + false, + null, + null + ), + NullHandling.replaceWithDefault() && canTestNumericNullsAsDefaultValues + ? ImmutableList.of("0", "3", "7") + : ImmutableList.of("0") + ); + } + + @Test + public void testVirtualNumericNullsAndZeros() + { + assertFilterMatches( + new RangeFilter( + "vd0", + ColumnType.DOUBLE, + 0.0, + 1.0, + false, + false, + null, + null + ), + canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "2", "7") : ImmutableList.of("0") + ); + + assertFilterMatches( + new RangeFilter( + "vf0", + ColumnType.FLOAT, + 0.0, + 1.0, + false, + false, + null, + null + ), + canTestNumericNullsAsDefaultValues ? ImmutableList.of("0", "4", "6") : ImmutableList.of("0") + ); + + assertFilterMatches( + new RangeFilter( + "vl0", + ColumnType.LONG, + 0L, + 1L, + false, + false, + null, + null + ), + NullHandling.replaceWithDefault() && canTestNumericNullsAsDefaultValues + ? ImmutableList.of("0", "3", "7") + : ImmutableList.of("0") + ); + } + + @Test + public void testNumericNulls() + { + assertFilterMatches( + new RangeFilter( + "f0", + ColumnType.FLOAT, + 1.0, + null, + false, + false, + null, + null + ), + ImmutableList.of("1", "2", "3", "5", "7") + ); + assertFilterMatches( + new RangeFilter( + "d0", + ColumnType.DOUBLE, + 1.0, + null, + false, + false, + null, + null + ), + ImmutableList.of("1", "3", "4", "5", "6") + ); + assertFilterMatches( + new RangeFilter( + "l0", + ColumnType.LONG, + 1L, + null, + false, + false, + null, + null + ), + ImmutableList.of("1", "2", "4", "5", "6") + ); + } + + @Test + public void testMatchWithExtractionFn() + { + String extractionJsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn superFn = new JavaScriptExtractionFn(extractionJsFn, false, JavaScriptConfig.getEnabledInstance()); + + String nullJsFn = "function(str) { return null; }"; + ExtractionFn makeNullFn = new JavaScriptExtractionFn(nullJsFn, false, JavaScriptConfig.getEnabledInstance()); + + assertFilterMatches( + new RangeFilter("dim0", ColumnType.STRING, "", "z", false, false, makeNullFn, null), + ImmutableList.of() + ); + + assertFilterMatches( + new RangeFilter( + "dim1", + ColumnType.STRING, + "super-ab", + "super-abd", + true, + true, + superFn, + null + ), + ImmutableList.of("5") + ); + + assertFilterMatches( + new RangeFilter("dim1", ColumnType.STRING, "super-0", "super-10", false, false, superFn, null), + ImmutableList.of("1", "3") + ); + + // auto schema ingests arrays instead of MVDs which aren't compatible with list filtered virtual column + if (!isAutoSchema()) { + assertFilterMatches( + new RangeFilter( + "dim2", + ColumnType.STRING, + "super-", + "super-zzzzzz", + false, + false, + superFn, + null + ), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + + if (NullHandling.replaceWithDefault()) { + assertFilterMatches( + new RangeFilter( + "dim2", + ColumnType.STRING, + "super-null", + "super-null", + false, + false, + superFn, + null + ), + ImmutableList.of("1", "2", "5") + ); + assertFilterMatches( + new RangeFilter( + "dim2", + ColumnType.STRING, + "super-null", + "super-null", + false, + false, + superFn, + null + ), + ImmutableList.of("1", "2", "5") + ); + } else { + assertFilterMatches( + new RangeFilter( + "dim2", + ColumnType.STRING, + "super-null", + "super-null", + false, + false, + superFn, + null + ), + ImmutableList.of("1", "5") + ); + assertFilterMatches( + new RangeFilter("dim2", ColumnType.STRING, "super-", "super-", false, false, superFn, null), + ImmutableList.of("2") + ); + assertFilterMatches( + new RangeFilter( + "dim2", + ColumnType.STRING, + "super-null", + "super-null", + false, + false, + superFn, + null + ), + ImmutableList.of("1", "5") + ); + assertFilterMatches( + new RangeFilter("dim2", ColumnType.STRING, "super-", "super-", false, false, superFn, null), + ImmutableList.of("2") + ); + } + } + + assertFilterMatches( + new RangeFilter( + "dim3", + ColumnType.STRING, + "super-null", + "super-null", + false, + false, + superFn, + null + ), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + + assertFilterMatches( + new RangeFilter( + "dim4", + ColumnType.STRING, + "super-null", + "super-null", + false, + false, + superFn, + null + ), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + + assertFilterMatches( + new RangeFilter("dim4", ColumnType.STRING, "super-null", "super-null", false, false, superFn, null), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); + } + + @Test + public void testListFilteredVirtualColumn() + { + assertFilterMatchesSkipVectorize( + new RangeFilter("allow-dim0", ColumnType.STRING, "0", "2", false, false, null, null), + ImmutableList.of() + ); + assertFilterMatchesSkipVectorize( + new RangeFilter("allow-dim0", ColumnType.STRING, "0", "6", false, false, null, null), + ImmutableList.of("3", "4") + ); + assertFilterMatchesSkipVectorize( + new RangeFilter("allow-dim0", ColumnType.STRING, null, "6", false, false, null, null), + ImmutableList.of("3", "4") + ); + + assertFilterMatchesSkipVectorize( + new RangeFilter("deny-dim0", ColumnType.STRING, "0", "6", false, false, null, null), + ImmutableList.of("0", "1", "2", "5", "6") + ); + assertFilterMatchesSkipVectorize( + new RangeFilter("deny-dim0", ColumnType.STRING, "3", "4", false, false, null, null), + ImmutableList.of() + ); + assertFilterMatchesSkipVectorize( + new RangeFilter("deny-dim0", ColumnType.STRING, null, "6", false, false, null, null), + ImmutableList.of("0", "1", "2", "5", "6") + ); + + if (isAutoSchema()) { + // bail out, auto ingests arrays instead of mvds and this virtual column is for mvd stuff + return; + } + + assertFilterMatchesSkipVectorize( + new RangeFilter("allow-dim2", ColumnType.STRING, "a", "c", false, false, null, null), + ImmutableList.of("0", "3", "6") + ); + assertFilterMatchesSkipVectorize( + new RangeFilter("allow-dim2", ColumnType.STRING, "c", "z", false, false, null, null), + ImmutableList.of() + ); + assertFilterMatchesSkipVectorize( + new RangeFilter("allow-dim2", ColumnType.STRING, null, "z", false, false, null, null), + ImmutableList.of("0", "3", "6") + ); + + assertFilterMatchesSkipVectorize( + new RangeFilter("deny-dim2", ColumnType.STRING, "a", "b", false, true, null, null), + ImmutableList.of() + ); + assertFilterMatchesSkipVectorize( + new RangeFilter("deny-dim2", ColumnType.STRING, "c", "z", false, false, null, null), + ImmutableList.of("4", "7") + ); + + assertFilterMatchesSkipVectorize( + new RangeFilter("deny-dim2", ColumnType.STRING, null, "z", false, false, null, null), + NullHandling.replaceWithDefault() ? ImmutableList.of("0", "4", "7") : ImmutableList.of("0", "2", "4", "7") + ); + } + + @Test + public void testRequiredColumnRewrite() + { + RangeFilter filter = new RangeFilter("dim0", ColumnType.STRING, "abc", "def", false, false, null, null); + RangeFilter filter2 = new RangeFilter("dim1", ColumnType.STRING, "abc", "def", false, false, null, null); + Assert.assertTrue(filter.supportsRequiredColumnRewrite()); + Assert.assertTrue(filter2.supportsRequiredColumnRewrite()); + + Filter rewrittenFilter = filter.rewriteRequiredColumns(ImmutableMap.of("dim0", "dim1")); + Assert.assertEquals(filter2, rewrittenFilter); + + Throwable t = Assert.assertThrows( + IAE.class, + () -> filter.rewriteRequiredColumns(ImmutableMap.of("invalidName", "dim1")) + ); + + Assert.assertEquals( + "Received a non-applicable rewrite: {invalidName=dim1}, filter's dimension: dim0", + t.getMessage() + ); + } + + @Test + public void testArrayRanges() + { + if (isAutoSchema()) { + // only auto schema supports array columns currently, this means the match value will need to be coerceable to + // the column value type... + + /* dim0 .. arrayString arrayLong arrayDouble + "0", .. ["a", "b", "c"], [1L, 2L, 3L], [1.1, 2.2, 3.3] + "1", .. [], [], [1.1, 2.2, 3.3] + "2", .. null, [1L, 2L, 3L], [null] + "3", .. ["a", "b", "c"], null, [] + "4", .. ["c", "d"], [null], [-1.1, -333.3] + "5", .. [null], [123L, 345L], null + "6", .. ["x", "y"], [100, 200], [1.1, null, 3.3] + "7", .. [null, "hello", "world"], [1234, 3456L, null], [1.23, 4.56, 6.78] + */ + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{"a", "b", "c"}, + new Object[]{"a", "b", "c"}, + false, + false, + null, + null + ), + ImmutableList.of("0", "3") + ); + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + null, + new Object[]{"a", "b", "c"}, + false, + false, + null, + null + ), + ImmutableList.of("0", "1", "3", "5", "7") + ); + + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{"a", "b", "c"}, + null, + true, + false, + null, + null + ), + ImmutableList.of("4", "6") + ); + + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + null, + new Object[]{"a", "b", "c"}, + false, + true, + null, + null + ), + ImmutableList.of("1", "5", "7") + ); + + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{"a", "b"}, + new Object[]{"a", "b", "c", "d"}, + true, + true, + null, + null + ), + ImmutableList.of("0", "3") + ); + + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{"c", "d"}, + new Object[]{"c", "d", "e"}, + false, + true, + null, + null + ), + ImmutableList.of("4") + ); + + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + null, + new Object[]{}, + false, + false, + null, + null + ), + ImmutableList.of("1") + ); + + assertFilterMatches( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + null, + new Object[]{null}, + false, + false, + null, + null + ), + ImmutableList.of("1", "5") + ); + + assertFilterMatches( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + null, + new Object[]{}, + false, + false, + null, + null + ), + ImmutableList.of("1") + ); + + assertFilterMatches( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{}, + null, + true, + false, + null, + null + ), + ImmutableList.of("0", "2", "4", "5", "6", "7") + ); + + assertFilterMatches( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + null, + new Object[]{null}, + false, + false, + null, + null + ), + ImmutableList.of("1", "4") + ); + + assertFilterMatches( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{1L, 2L, 3L}, + new Object[]{1L, 2L, 3L}, + false, + false, + null, + null + ), + ImmutableList.of("0", "2") + ); + + + assertFilterMatches( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + null, + new Object[]{1L, 2L, 3L}, + false, + true, + null, + null + ), + ImmutableList.of("1", "4") + ); + + assertFilterMatches( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{1L, 2L, 3L}, + null, + true, + false, + null, + null + ), + ImmutableList.of("5", "6", "7") + ); + + // empties and nulls still sort before numbers + assertFilterMatches( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + null, + new Object[]{-1L}, + false, + false, + null, + null + ), + ImmutableList.of("1", "4") + ); + + assertFilterMatches( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + null, + new Object[]{}, + false, + false, + null, + null + ), + ImmutableList.of("3") + ); + + assertFilterMatches( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + new Object[]{}, + null, + true, + false, + null, + null + ), + ImmutableList.of("0", "1", "2", "4", "6", "7") + ); + + assertFilterMatches( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + null, + new Object[]{null}, + false, + false, + null, + null + ), + ImmutableList.of("2", "3") + ); + + assertFilterMatches( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + new Object[]{1.1, 2.2, 3.3}, + new Object[]{1.1, 2.2, 3.3}, + false, + false, + null, + null + ), + ImmutableList.of("0", "1") + ); + assertFilterMatches( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + new Object[]{1.1, 2.2, 3.3}, + null, + true, + false, + null, + null + ), + ImmutableList.of("7") + ); + + assertFilterMatches( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + null, + new Object[]{1.1, 2.2, 3.3}, + true, + false, + null, + null + ), + ImmutableList.of("0", "1", "2", "3", "4", "6") + ); + + // empties and nulls sort before numbers + assertFilterMatches( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + null, + new Object[]{0.0}, + true, + false, + null, + null + ), + ImmutableList.of("2", "3", "4") + ); + } + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(RangeFilter.class) + .withNonnullFields("column", "matchValueType") + .withIgnoredFields( + "matchValueExpressionType", + "lowerEval", + "upperEval", + "cachedOptimizedFilter", + "stringPredicateSupplier", + "longPredicateSupplier", + "floatPredicateSupplier", + "doublePredicateSupplier", + "arrayPredicates", + "typeDetectingArrayPredicateSupplier" + ) + .withPrefabValues(ColumnType.class, ColumnType.STRING, ColumnType.DOUBLE) + .usingGetClass() + .verify(); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java index 30f0c6b7170e..a6ec01681771 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RegexFilterTest.java @@ -94,6 +94,9 @@ public void testSingleValueStringColumnWithNulls() @Test public void testMultiValueStringColumn() { + if (isAutoSchema()) { + return; + } if (NullHandling.replaceWithDefault()) { assertFilterMatches(new RegexDimFilter("dim2", ".*", null), ImmutableList.of("0", "3", "4")); } else { @@ -130,14 +133,17 @@ public void testRegexWithExtractionFn() ExtractionFn changeNullFn = new JavaScriptExtractionFn(nullJsFn, false, JavaScriptConfig.getEnabledInstance()); if (NullHandling.replaceWithDefault()) { assertFilterMatches(new RegexDimFilter("dim1", ".*ANYMORE", changeNullFn), ImmutableList.of("0")); - assertFilterMatches(new RegexDimFilter("dim2", ".*ANYMORE", changeNullFn), ImmutableList.of("1", "2", "5")); + assertFilterMatchesSkipArrays( + new RegexDimFilter("dim2", ".*ANYMORE", changeNullFn), + ImmutableList.of("1", "2", "5") + ); } else { assertFilterMatches(new RegexDimFilter("dim1", ".*ANYMORE", changeNullFn), ImmutableList.of()); - assertFilterMatches(new RegexDimFilter("dim2", ".*ANYMORE", changeNullFn), ImmutableList.of("1", "5")); + assertFilterMatchesSkipArrays(new RegexDimFilter("dim2", ".*ANYMORE", changeNullFn), ImmutableList.of("1", "5")); } assertFilterMatches(new RegexDimFilter("dim1", "ab.*", changeNullFn), ImmutableList.of("4", "5")); - assertFilterMatches(new RegexDimFilter("dim2", "a.*", changeNullFn), ImmutableList.of("0", "3")); + assertFilterMatchesSkipArrays(new RegexDimFilter("dim2", "a.*", changeNullFn), ImmutableList.of("0", "3")); assertFilterMatches(new RegexDimFilter("dim3", ".*ANYMORE", changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5")); assertFilterMatches(new RegexDimFilter("dim3", "a.*", changeNullFn), ImmutableList.of()); diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java index a7246642925f..8aef0ef27ab9 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SearchQueryFilterTest.java @@ -107,6 +107,9 @@ public void testSingleValueStringColumnWithNulls() @Test public void testMultiValueStringColumn() { + if (isAutoSchema()) { + return; + } if (NullHandling.replaceWithDefault()) { assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue(""), null), ImmutableList.of("0", "3", "4")); } else { @@ -152,7 +155,7 @@ public void testSearchQueryWithExtractionFn() new SearchQueryDimFilter("dim1", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("0") ); - assertFilterMatches( + assertFilterMatchesSkipArrays( new SearchQueryDimFilter("dim2", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("1", "2", "5") ); @@ -162,7 +165,7 @@ public void testSearchQueryWithExtractionFn() new SearchQueryDimFilter("dim1", specForValue("ANYMORE"), changeNullFn), ImmutableList.of() ); - assertFilterMatches( + assertFilterMatchesSkipArrays( new SearchQueryDimFilter("dim2", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("1", "5") ); @@ -174,12 +177,21 @@ public void testSearchQueryWithExtractionFn() ); assertFilterMatches(new SearchQueryDimFilter("dim1", specForValue("ab"), changeNullFn), ImmutableList.of("4", "5")); - assertFilterMatches(new SearchQueryDimFilter("dim2", specForValue("a"), changeNullFn), ImmutableList.of("0", "3")); + assertFilterMatchesSkipArrays( + new SearchQueryDimFilter("dim2", specForValue("a"), changeNullFn), + ImmutableList.of("0", "3") + ); - assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5")); + assertFilterMatches( + new SearchQueryDimFilter("dim3", specForValue("ANYMORE"), changeNullFn), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); assertFilterMatches(new SearchQueryDimFilter("dim3", specForValue("a"), changeNullFn), ImmutableList.of()); - assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5")); + assertFilterMatches( + new SearchQueryDimFilter("dim4", specForValue("ANYMORE"), changeNullFn), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("a"), changeNullFn), ImmutableList.of()); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java index 95765949ac0f..80b1bb1a88d1 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/SelectorFilterTest.java @@ -120,13 +120,22 @@ public void testListFilteredVirtualColumn() { assertFilterMatchesSkipVectorize(new SelectorDimFilter("allow-dim0", "1", null), ImmutableList.of()); assertFilterMatchesSkipVectorize(new SelectorDimFilter("allow-dim0", "4", null), ImmutableList.of("4")); - assertFilterMatchesSkipVectorize(new SelectorDimFilter("allow-dim0", null, null), ImmutableList.of("0", "1", "2", "5")); + assertFilterMatchesSkipVectorize( + new SelectorDimFilter("allow-dim0", null, null), + ImmutableList.of("0", "1", "2", "5") + ); assertFilterMatchesSkipVectorize(new SelectorDimFilter("deny-dim0", "0", null), ImmutableList.of("0")); assertFilterMatchesSkipVectorize(new SelectorDimFilter("deny-dim0", "4", null), ImmutableList.of()); assertFilterMatchesSkipVectorize(new SelectorDimFilter("deny-dim0", null, null), ImmutableList.of("3", "4")); + if (isAutoSchema()) { + return; + } assertFilterMatchesSkipVectorize(new SelectorDimFilter("allow-dim2", "b", null), ImmutableList.of()); assertFilterMatchesSkipVectorize(new SelectorDimFilter("allow-dim2", "a", null), ImmutableList.of("0", "3")); - assertFilterMatchesSkipVectorize(new SelectorDimFilter("allow-dim2", null, null), ImmutableList.of("1", "2", "4", "5")); + assertFilterMatchesSkipVectorize( + new SelectorDimFilter("allow-dim2", null, null), + ImmutableList.of("1", "2", "4", "5") + ); assertFilterMatchesSkipVectorize(new SelectorDimFilter("deny-dim2", "b", null), ImmutableList.of("0")); assertFilterMatchesSkipVectorize(new SelectorDimFilter("deny-dim2", "a", null), ImmutableList.of()); if (NullHandling.replaceWithDefault()) { @@ -181,6 +190,9 @@ public void testSingleValueVirtualStringColumnWithNulls() @Test public void testMultiValueStringColumn() { + if (isAutoSchema()) { + return; + } if (NullHandling.replaceWithDefault()) { assertFilterMatches(new SelectorDimFilter("dim2", null, null), ImmutableList.of("1", "2", "5")); assertFilterMatches(new SelectorDimFilter("dim2", "", null), ImmutableList.of("1", "2", "5")); @@ -250,8 +262,11 @@ public void testSelectorWithLookupExtractionFn() assertFilterMatches(new SelectorDimFilter("dim1", "HELLO", lookupFn), ImmutableList.of("3", "4")); assertFilterMatches(new SelectorDimFilter("dim1", "UNKNOWN", lookupFn), ImmutableList.of("0", "1", "2", "5")); - assertFilterMatches(new SelectorDimFilter("dim2", "HELLO", lookupFn), ImmutableList.of("0", "3")); - assertFilterMatches(new SelectorDimFilter("dim2", "UNKNOWN", lookupFn), ImmutableList.of("0", "1", "2", "4", "5")); + assertFilterMatchesSkipArrays(new SelectorDimFilter("dim2", "HELLO", lookupFn), ImmutableList.of("0", "3")); + assertFilterMatchesSkipArrays( + new SelectorDimFilter("dim2", "UNKNOWN", lookupFn), + ImmutableList.of("0", "1", "2", "4", "5") + ); assertFilterMatches(new SelectorDimFilter("dim3", "HELLO", lookupFn), ImmutableList.of()); assertFilterMatches( diff --git a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java index ae6622d1d77e..6cec0392dd3c 100644 --- a/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapterTest.java @@ -63,11 +63,11 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.VirtualColumns; -import org.apache.druid.segment.column.AllTrueBitmapColumnIndex; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.SelectorFilter; +import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.joda.time.DateTime; import org.joda.time.Interval; 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 a5202260efee..b80f4a1e2ec1 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 @@ -53,6 +53,7 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -60,6 +61,7 @@ import org.apache.druid.segment.join.table.RowBasedIndexedTable; import org.junit.Assert; +import javax.annotation.Nullable; import java.io.File; import java.io.IOException; import java.io.InputStream; @@ -152,6 +154,15 @@ public Supplier makeLongProcessor(BaseLongColumnValueSelector selector) return () -> NullHandling.sqlCompatible() && selector.isNull() ? null : selector.getLong(); } + @Override + public Supplier makeArrayProcessor( + BaseObjectColumnValueSelector selector, + @Nullable ColumnCapabilities columnCapabilities + ) + { + return selector::getObject; + } + @Override public Supplier makeComplexProcessor(BaseObjectColumnValueSelector selector) { diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index d6131b2d820a..80daa3549dcf 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -53,11 +53,11 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.serde.ColumnPartSerde; import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; import org.apache.druid.segment.vector.BitmapVectorOffset; @@ -263,7 +263,6 @@ public void testBasicFunctionality() throws IOException final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); - Assert.assertTrue(capabilities.isFilterable()); Assert.assertTrue(holder.getColumnFormat() instanceof NestedCommonFormatColumn.Format); try (NestedDataComplexColumn column = (NestedDataComplexColumn) holder.getColumn()) { smokeTest(column); @@ -287,7 +286,6 @@ public void testArrayFunctionality() throws IOException final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); - Assert.assertTrue(capabilities.isFilterable()); Assert.assertTrue(holder.getColumnFormat() instanceof NestedCommonFormatColumn.Format); try (NestedDataComplexColumn column = (NestedDataComplexColumn) holder.getColumn()) { smokeTestArrays(column); @@ -352,8 +350,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector xDimSelector = column.makeDimensionSelector(xPath, offset, null); ColumnIndexSupplier xIndexSupplier = column.getColumnIndexSupplier(xPath); Assert.assertNotNull(xIndexSupplier); - StringValueSetIndex xValueIndex = xIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex xPredicateIndex = xIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes xValueIndex = xIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes xPredicateIndex = xIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex xNulls = xIndexSupplier.as(NullValueIndex.class); final List yPath = NestedPathFinder.parseJsonPath("$.y"); @@ -363,8 +361,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector yDimSelector = column.makeDimensionSelector(yPath, offset, null); ColumnIndexSupplier yIndexSupplier = column.getColumnIndexSupplier(yPath); Assert.assertNotNull(yIndexSupplier); - StringValueSetIndex yValueIndex = yIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex yPredicateIndex = yIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes yValueIndex = yIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes yPredicateIndex = yIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex yNulls = yIndexSupplier.as(NullValueIndex.class); final List zPath = NestedPathFinder.parseJsonPath("$.z"); @@ -374,8 +372,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector zDimSelector = column.makeDimensionSelector(zPath, offset, null); ColumnIndexSupplier zIndexSupplier = column.getColumnIndexSupplier(zPath); Assert.assertNotNull(zIndexSupplier); - StringValueSetIndex zValueIndex = zIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex zPredicateIndex = zIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes zValueIndex = zIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes zPredicateIndex = zIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex zNulls = zIndexSupplier.as(NullValueIndex.class); final List vPath = NestedPathFinder.parseJsonPath("$.v"); @@ -388,8 +386,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector vDimSelector = column.makeDimensionSelector(vPath, offset, null); ColumnIndexSupplier vIndexSupplier = column.getColumnIndexSupplier(vPath); Assert.assertNotNull(vIndexSupplier); - StringValueSetIndex vValueIndex = vIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex vPredicateIndex = vIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes vValueIndex = vIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes vPredicateIndex = vIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex vNulls = vIndexSupplier.as(NullValueIndex.class); final List nullishPath = NestedPathFinder.parseJsonPath("$.nullish"); @@ -399,8 +397,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector nullishDimSelector = column.makeDimensionSelector(nullishPath, offset, null); ColumnIndexSupplier nullishIndexSupplier = column.getColumnIndexSupplier(nullishPath); Assert.assertNotNull(nullishIndexSupplier); - StringValueSetIndex nullishValueIndex = nullishIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex nullishPredicateIndex = nullishIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes nullishValueIndex = nullishIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes nullishPredicateIndex = nullishIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex nullishNulls = nullishIndexSupplier.as(NullValueIndex.class); Assert.assertEquals(ImmutableList.of(nullishPath, vPath, xPath, yPath, zPath), column.getNestedFields()); @@ -461,8 +459,8 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException VectorObjectSelector sVectorSelectorFiltered = column.makeVectorObjectSelector(sPath, bitmapVectorOffset); ColumnIndexSupplier sIndexSupplier = column.getColumnIndexSupplier(sPath); Assert.assertNotNull(sIndexSupplier); - Assert.assertNull(sIndexSupplier.as(StringValueSetIndex.class)); - Assert.assertNull(sIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(sIndexSupplier.as(StringValueSetIndexes.class)); + Assert.assertNull(sIndexSupplier.as(DruidPredicateIndexes.class)); NullValueIndex sNulls = sIndexSupplier.as(NullValueIndex.class); final List sElementPath = NestedPathFinder.parseJsonPath("$.s[1]"); @@ -474,8 +472,8 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException ); ColumnIndexSupplier sElementIndexSupplier = column.getColumnIndexSupplier(sElementPath); Assert.assertNotNull(sElementIndexSupplier); - Assert.assertNull(sElementIndexSupplier.as(StringValueSetIndex.class)); - Assert.assertNull(sElementIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(sElementIndexSupplier.as(StringValueSetIndexes.class)); + Assert.assertNull(sElementIndexSupplier.as(DruidPredicateIndexes.class)); Assert.assertNull(sElementIndexSupplier.as(NullValueIndex.class)); final List lPath = NestedPathFinder.parseJsonPath("$.l"); @@ -486,8 +484,8 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException VectorObjectSelector lVectorSelectorFiltered = column.makeVectorObjectSelector(lPath, bitmapVectorOffset); ColumnIndexSupplier lIndexSupplier = column.getColumnIndexSupplier(lPath); Assert.assertNotNull(lIndexSupplier); - Assert.assertNull(lIndexSupplier.as(StringValueSetIndex.class)); - Assert.assertNull(lIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(lIndexSupplier.as(StringValueSetIndexes.class)); + Assert.assertNull(lIndexSupplier.as(DruidPredicateIndexes.class)); NullValueIndex lNulls = lIndexSupplier.as(NullValueIndex.class); final List lElementPath = NestedPathFinder.parseJsonPath("$.l[1]"); @@ -500,8 +498,8 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException ); ColumnIndexSupplier lElementIndexSupplier = column.getColumnIndexSupplier(lElementPath); Assert.assertNotNull(lElementIndexSupplier); - Assert.assertNull(lElementIndexSupplier.as(StringValueSetIndex.class)); - Assert.assertNull(lElementIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(lElementIndexSupplier.as(StringValueSetIndexes.class)); + Assert.assertNull(lElementIndexSupplier.as(DruidPredicateIndexes.class)); Assert.assertNull(lElementIndexSupplier.as(NullValueIndex.class)); final List dPath = NestedPathFinder.parseJsonPath("$.d"); @@ -512,8 +510,8 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException VectorObjectSelector dVectorSelectorFiltered = column.makeVectorObjectSelector(dPath, bitmapVectorOffset); ColumnIndexSupplier dIndexSupplier = column.getColumnIndexSupplier(dPath); Assert.assertNotNull(dIndexSupplier); - Assert.assertNull(dIndexSupplier.as(StringValueSetIndex.class)); - Assert.assertNull(dIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(dIndexSupplier.as(StringValueSetIndexes.class)); + Assert.assertNull(dIndexSupplier.as(DruidPredicateIndexes.class)); NullValueIndex dNulls = dIndexSupplier.as(NullValueIndex.class); final List dElementPath = NestedPathFinder.parseJsonPath("$.d[1]"); @@ -526,14 +524,14 @@ private void smokeTestArrays(NestedDataComplexColumn column) throws IOException ); ColumnIndexSupplier dElementIndexSupplier = column.getColumnIndexSupplier(dElementPath); Assert.assertNotNull(dElementIndexSupplier); - Assert.assertNull(dElementIndexSupplier.as(StringValueSetIndex.class)); - Assert.assertNull(dElementIndexSupplier.as(DruidPredicateIndex.class)); + Assert.assertNull(dElementIndexSupplier.as(StringValueSetIndexes.class)); + Assert.assertNull(dElementIndexSupplier.as(DruidPredicateIndexes.class)); Assert.assertNull(dElementIndexSupplier.as(NullValueIndex.class)); - ImmutableBitmap sNullIndex = sNulls.forNull().computeBitmapResult(resultFactory); - ImmutableBitmap lNullIndex = lNulls.forNull().computeBitmapResult(resultFactory); - ImmutableBitmap dNullIndex = dNulls.forNull().computeBitmapResult(resultFactory); + ImmutableBitmap sNullIndex = sNulls.get().computeBitmapResult(resultFactory); + ImmutableBitmap lNullIndex = lNulls.get().computeBitmapResult(resultFactory); + ImmutableBitmap dNullIndex = dNulls.get().computeBitmapResult(resultFactory); int rowCounter = 0; while (offset.withinBounds()) { @@ -681,8 +679,8 @@ private void testPath( String path, ColumnValueSelector valueSelector, DimensionSelector dimSelector, - StringValueSetIndex valueSetIndex, - DruidPredicateIndex predicateIndex, + StringValueSetIndexes valueSetIndex, + DruidPredicateIndexes predicateIndex, NullValueIndex nullValueIndex, @Nullable ColumnType singleType ) @@ -723,7 +721,7 @@ private void testPath( Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) .computeBitmapResult(resultFactory) .get(rowNumber)); - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches()); Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); @@ -737,7 +735,7 @@ private void testPath( Assert.assertNull(dimSelector.getObject()); Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0))); - Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) .computeBitmapResult(resultFactory) diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index 154be02dfe77..02b634c1f8a6 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -46,16 +46,16 @@ import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.column.TypeStrategy; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.serde.ColumnPartSerde; import org.apache.druid.segment.serde.ComplexColumnPartSerde; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -229,7 +229,6 @@ public void testBasicFunctionality() throws IOException final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); - Assert.assertTrue(capabilities.isFilterable()); Assert.assertTrue(holder.getColumnFormat() instanceof NestedDataComplexTypeSerde.NestedColumnFormatV4); try (NestedDataComplexColumn column = (NestedDataComplexColumn) holder.getColumn()) { smokeTest(column); @@ -313,7 +312,7 @@ public void testLegacyV3ReaderFormat() throws IOException ColumnIndexSupplier indexSupplier = v3.getColumnIndexSupplier(path); Assert.assertNotNull(indexSupplier); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); BitmapColumnIndex indexForValue = valueSetIndex.forValue(firstValue); @@ -357,7 +356,7 @@ public void testLegacyV4ReaderFormat() throws IOException ColumnIndexSupplier indexSupplier = v4.getColumnIndexSupplier(path); Assert.assertNotNull(indexSupplier); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); BitmapColumnIndex indexForValue = valueSetIndex.forValue(firstValue); @@ -378,8 +377,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector xDimSelector = column.makeDimensionSelector(xPath, offset, null); ColumnIndexSupplier xIndexSupplier = column.getColumnIndexSupplier(xPath); Assert.assertNotNull(xIndexSupplier); - StringValueSetIndex xValueIndex = xIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex xPredicateIndex = xIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes xValueIndex = xIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes xPredicateIndex = xIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex xNulls = xIndexSupplier.as(NullValueIndex.class); final List yPath = NestedPathFinder.parseJsonPath("$.y"); @@ -389,8 +388,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector yDimSelector = column.makeDimensionSelector(yPath, offset, null); ColumnIndexSupplier yIndexSupplier = column.getColumnIndexSupplier(yPath); Assert.assertNotNull(yIndexSupplier); - StringValueSetIndex yValueIndex = yIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex yPredicateIndex = yIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes yValueIndex = yIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes yPredicateIndex = yIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex yNulls = yIndexSupplier.as(NullValueIndex.class); final List zPath = NestedPathFinder.parseJsonPath("$.z"); @@ -400,8 +399,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector zDimSelector = column.makeDimensionSelector(zPath, offset, null); ColumnIndexSupplier zIndexSupplier = column.getColumnIndexSupplier(zPath); Assert.assertNotNull(zIndexSupplier); - StringValueSetIndex zValueIndex = zIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex zPredicateIndex = zIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes zValueIndex = zIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes zPredicateIndex = zIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex zNulls = zIndexSupplier.as(NullValueIndex.class); final List vPath = NestedPathFinder.parseJsonPath("$.v"); @@ -414,8 +413,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector vDimSelector = column.makeDimensionSelector(vPath, offset, null); ColumnIndexSupplier vIndexSupplier = column.getColumnIndexSupplier(vPath); Assert.assertNotNull(vIndexSupplier); - StringValueSetIndex vValueIndex = vIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex vPredicateIndex = vIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes vValueIndex = vIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes vPredicateIndex = vIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex vNulls = vIndexSupplier.as(NullValueIndex.class); final List nullishPath = NestedPathFinder.parseJsonPath("$.nullish"); @@ -425,8 +424,8 @@ private void smokeTest(NestedDataComplexColumn column) throws IOException DimensionSelector nullishDimSelector = column.makeDimensionSelector(nullishPath, offset, null); ColumnIndexSupplier nullishIndexSupplier = column.getColumnIndexSupplier(nullishPath); Assert.assertNotNull(nullishIndexSupplier); - StringValueSetIndex nullishValueIndex = nullishIndexSupplier.as(StringValueSetIndex.class); - DruidPredicateIndex nullishPredicateIndex = nullishIndexSupplier.as(DruidPredicateIndex.class); + StringValueSetIndexes nullishValueIndex = nullishIndexSupplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes nullishPredicateIndex = nullishIndexSupplier.as(DruidPredicateIndexes.class); NullValueIndex nullishNulls = nullishIndexSupplier.as(NullValueIndex.class); Assert.assertEquals(ImmutableList.of(nullishPath, vPath, xPath, yPath, zPath), column.getNestedFields()); @@ -464,8 +463,8 @@ private void testPath( String path, ColumnValueSelector valueSelector, DimensionSelector dimSelector, - StringValueSetIndex valueSetIndex, - DruidPredicateIndex predicateIndex, + StringValueSetIndexes valueSetIndex, + DruidPredicateIndexes predicateIndex, NullValueIndex nullValueIndex, @Nullable ColumnType singleType ) @@ -506,7 +505,7 @@ private void testPath( Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) .computeBitmapResult(resultFactory) .get(rowNumber)); - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches()); Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); @@ -521,7 +520,7 @@ private void testPath( Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0))); Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) .computeBitmapResult(resultFactory) .get(rowNumber)); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java index 9503e55cc574..fffb9068a3ef 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java @@ -28,17 +28,8 @@ import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.InDimFilter; -import org.apache.druid.segment.column.BitmapColumnIndex; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; -import org.apache.druid.segment.column.DictionaryEncodedValueIndex; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.LexicographicalRangeIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.NumericRangeIndex; -import org.apache.druid.segment.column.SpatialIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.column.TypeStrategies; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.FixedIndexed; @@ -47,6 +38,15 @@ import org.apache.druid.segment.data.GenericIndexedWriter; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.NumericRangeIndexes; +import org.apache.druid.segment.index.semantic.SpatialIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.serde.Serializer; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -168,7 +168,7 @@ public void testSingleTypeStringColumnValueIndex() throws IOException // local: [b, foo, fooo, z] // column: [foo, b, fooo, b, z, fooo, z, b, b, foo] - BitmapColumnIndex columnIndex = nullIndex.forNull(); + BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); @@ -180,7 +180,7 @@ public void testSingleTypeStringColumnValueSetIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // 10 rows @@ -213,7 +213,7 @@ public void testSingleTypeStringColumnRangeIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); - LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); + LexicographicalRangeIndexes rangeIndex = indexSupplier.as(LexicographicalRangeIndexes.class); Assert.assertNotNull(rangeIndex); // 10 rows @@ -370,7 +370,7 @@ public void testSingleTypeStringColumnRangeIndexWithPredicate() throws IOExcepti { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); - LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); + LexicographicalRangeIndexes rangeIndex = indexSupplier.as(LexicographicalRangeIndexes.class); Assert.assertNotNull(rangeIndex); // 10 rows @@ -438,7 +438,7 @@ public void testSingleTypeStringColumnPredicateIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringSupplier(); - DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = indexSupplier.as(DruidPredicateIndexes.class); Assert.assertNotNull(predicateIndex); DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( null, @@ -468,7 +468,7 @@ public void testSingleTypeStringColumnWithNullValueIndex() throws IOException // local: [null, b, foo, fooo, z] // column: [foo, null, fooo, b, z, fooo, z, null, null, foo] - BitmapColumnIndex columnIndex = nullIndex.forNull(); + BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); @@ -480,7 +480,7 @@ public void testSingleTypeStringColumnWithNullValueSetIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // 10 rows @@ -513,7 +513,7 @@ public void testSingleValueStringWithNullRangeIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); - LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); + LexicographicalRangeIndexes rangeIndex = indexSupplier.as(LexicographicalRangeIndexes.class); Assert.assertNotNull(rangeIndex); // 10 rows @@ -603,7 +603,7 @@ public void testSingleValueStringWithNullPredicateIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeStringWithNullsSupplier(); - DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = indexSupplier.as(DruidPredicateIndexes.class); Assert.assertNotNull(predicateIndex); DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( null, @@ -626,7 +626,7 @@ public void testSingleTypeLongColumnValueSetIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // sanity check to make sure we don't return indexes we don't support @@ -655,65 +655,65 @@ public void testSingleTypeLongColumnRangeIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); - NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); - Assert.assertNotNull(rangeIndex); + NumericRangeIndexes rangeIndexes = indexSupplier.as(NumericRangeIndexes.class); + Assert.assertNotNull(rangeIndexes); // 10 rows // local: [1, 3, 100, 300] // column: [100, 1, 300, 1, 3, 3, 100, 300, 300, 1] - BitmapColumnIndex forRange = rangeIndex.forRange(10L, true, 400L, true); + BitmapColumnIndex forRange = rangeIndexes.forRange(10L, true, 400L, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 2, 6, 7, 8); - forRange = rangeIndex.forRange(1, true, 3, true); + forRange = rangeIndexes.forRange(1, true, 3, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(1, false, 3, true); + forRange = rangeIndexes.forRange(1, false, 3, true); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 1, 3, 9); - forRange = rangeIndex.forRange(1, false, 3, false); + forRange = rangeIndexes.forRange(1, false, 3, false); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 1, 3, 4, 5, 9); - forRange = rangeIndex.forRange(100L, true, 300L, true); + forRange = rangeIndexes.forRange(100L, true, 300L, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(100L, true, 300L, false); + forRange = rangeIndexes.forRange(100L, true, 300L, false); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 2, 7, 8); - forRange = rangeIndex.forRange(100L, false, 300L, true); + forRange = rangeIndexes.forRange(100L, false, 300L, true); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 6); - forRange = rangeIndex.forRange(100L, false, 300L, false); + forRange = rangeIndexes.forRange(100L, false, 300L, false); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 2, 6, 7, 8); - forRange = rangeIndex.forRange(null, true, null, true); + forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); - forRange = rangeIndex.forRange(null, false, null, false); + forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); @@ -724,7 +724,7 @@ public void testSingleTypeLongColumnPredicateIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplier(); - DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = indexSupplier.as(DruidPredicateIndexes.class); Assert.assertNotNull(predicateIndex); DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( null, @@ -754,7 +754,7 @@ public void testSingleTypeLongColumnWithNullValueIndex() throws IOException // local: [null, 1, 3, 100, 300] // column: [100, 1, null, 1, 3, null, 100, 300, null, 1] - BitmapColumnIndex columnIndex = nullIndex.forNull(); + BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); @@ -766,7 +766,7 @@ public void testSingleTypeLongColumnWithNullValueSetIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // 10 rows @@ -811,61 +811,61 @@ public void testSingleValueLongWithNullRangeIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); - NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); - Assert.assertNotNull(rangeIndex); + NumericRangeIndexes rangeIndexes = indexSupplier.as(NumericRangeIndexes.class); + Assert.assertNotNull(rangeIndexes); // 10 rows // local: [null, 1, 3, 100, 300] // column: [100, 1, null, 1, 3, null, 100, 300, null, 1] - BitmapColumnIndex forRange = rangeIndex.forRange(100, false, 700, true); + BitmapColumnIndex forRange = rangeIndexes.forRange(100, false, 700, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 6, 7); - forRange = rangeIndex.forRange(100, true, 300, true); + forRange = rangeIndexes.forRange(100, true, 300, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(100, false, 300, true); + forRange = rangeIndexes.forRange(100, false, 300, true); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 6); - forRange = rangeIndex.forRange(100, true, 300, false); + forRange = rangeIndexes.forRange(100, true, 300, false); Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 7); - forRange = rangeIndex.forRange(100, false, 300, false); + forRange = rangeIndexes.forRange(100, false, 300, false); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 6, 7); - forRange = rangeIndex.forRange(null, true, null, true); + forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 9); - forRange = rangeIndex.forRange(null, false, null, false); + forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 9); - forRange = rangeIndex.forRange(null, false, 0, false); + forRange = rangeIndexes.forRange(null, false, 0, false); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(null, false, 1, false); + forRange = rangeIndexes.forRange(null, false, 1, false); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 1, 3, 9); - forRange = rangeIndex.forRange(null, false, 1, true); + forRange = rangeIndexes.forRange(null, false, 1, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); @@ -876,7 +876,7 @@ public void testSingleValueLongWithNullPredicateIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeLongSupplierWithNull(); - DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = indexSupplier.as(DruidPredicateIndexes.class); Assert.assertNotNull(predicateIndex); DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( null, @@ -899,7 +899,7 @@ public void testSingleTypeDoubleColumnValueSetIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // sanity check to make sure we don't return indexes we don't support @@ -928,87 +928,87 @@ public void testSingleTypeDoubleColumnRangeIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); - NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); - Assert.assertNotNull(rangeIndex); + NumericRangeIndexes rangeIndexes = indexSupplier.as(NumericRangeIndexes.class); + Assert.assertNotNull(rangeIndexes); // 10 rows // local: [1.1, 1.2, 3.3, 6.6] // column: [1.1, 1.1, 1.2, 3.3, 1.2, 6.6, 3.3, 1.2, 1.1, 3.3] - BitmapColumnIndex forRange = rangeIndex.forRange(1.0, true, 5.0, true); + BitmapColumnIndex forRange = rangeIndexes.forRange(1.0, true, 5.0, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.9, forRange.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 2, 3, 4, 6, 7, 8, 9); - forRange = rangeIndex.forRange(1.1, false, 3.3, false); + forRange = rangeIndexes.forRange(1.1, false, 3.3, false); Assert.assertNotNull(forRange); Assert.assertEquals(0.9, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 2, 3, 4, 6, 7, 8, 9); - forRange = rangeIndex.forRange(1.1, true, 3.3, true); + forRange = rangeIndexes.forRange(1.1, true, 3.3, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 2, 4, 7); - forRange = rangeIndex.forRange(null, true, null, true); + forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); - forRange = rangeIndex.forRange(null, false, null, false); + forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); - forRange = rangeIndex.forRange(1.111, true, 1.19, true); + forRange = rangeIndexes.forRange(1.111, true, 1.19, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(1.01, true, 1.09, true); + forRange = rangeIndexes.forRange(1.01, true, 1.09, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(0.05, true, 0.98, true); + forRange = rangeIndexes.forRange(0.05, true, 0.98, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(0.05, true, 1.1, true); + forRange = rangeIndexes.forRange(0.05, true, 1.1, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(8.99, true, 10.10, true); + forRange = rangeIndexes.forRange(8.99, true, 10.10, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(8.99, true, 10.10, true); + forRange = rangeIndexes.forRange(8.99, true, 10.10, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(10.00, true, 10.10, true); + forRange = rangeIndexes.forRange(10.00, true, 10.10, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); @@ -1021,7 +1021,7 @@ public void testSingleTypeDoubleColumnPredicateIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplier(); - DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = indexSupplier.as(DruidPredicateIndexes.class); Assert.assertNotNull(predicateIndex); DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( null, @@ -1051,7 +1051,7 @@ public void testSingleTypeDoubleColumnWithNullValueIndex() throws IOException // local: [null, 1.1, 1.2, 3.3, 6.6] // column: [1.1, null, 1.2, null, 1.2, 6.6, null, 1.2, 1.1, 3.3] - BitmapColumnIndex columnIndex = nullIndex.forNull(); + BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); @@ -1063,7 +1063,7 @@ public void testSingleTypeDoubleColumnWithNullValueSetIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // 10 rows @@ -1108,46 +1108,46 @@ public void testSingleValueDoubleWithNullRangeIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); - NumericRangeIndex rangeIndex = indexSupplier.as(NumericRangeIndex.class); - Assert.assertNotNull(rangeIndex); + NumericRangeIndexes rangeIndexes = indexSupplier.as(NumericRangeIndexes.class); + Assert.assertNotNull(rangeIndexes); // 10 rows // local: [null, 1.1, 1.2, 3.3, 6.6] // column: [1.1, null, 1.2, null, 1.2, 6.6, null, 1.2, 1.1, 3.3] - BitmapColumnIndex forRange = rangeIndex.forRange(1.1, false, 5.0, true); + BitmapColumnIndex forRange = rangeIndexes.forRange(1.1, false, 5.0, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 2, 4, 7, 8, 9); - forRange = rangeIndex.forRange(null, true, null, true); + forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 2, 4, 5, 7, 8, 9); - forRange = rangeIndex.forRange(null, false, null, false); + forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 2, 4, 5, 7, 8, 9); - forRange = rangeIndex.forRange(null, true, 1.0, true); + forRange = rangeIndexes.forRange(null, true, 1.0, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); - forRange = rangeIndex.forRange(null, true, 1.1, false); + forRange = rangeIndexes.forRange(null, true, 1.1, false); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 0, 8); - forRange = rangeIndex.forRange(6.6, false, null, false); + forRange = rangeIndexes.forRange(6.6, false, null, false); Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap, 5); - forRange = rangeIndex.forRange(6.6, true, null, false); + forRange = rangeIndexes.forRange(6.6, true, null, false); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); bitmap = forRange.computeBitmapResult(bitmapResultFactory); checkBitmap(bitmap); @@ -1158,7 +1158,7 @@ public void testSingleValueDoubleWithNullPredicateIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeSingleTypeDoubleSupplierWithNull(); - DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = indexSupplier.as(DruidPredicateIndexes.class); Assert.assertNotNull(predicateIndex); DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( null, @@ -1191,7 +1191,7 @@ public void testVariantNullValueIndex() throws IOException // local: [null, b, z, 1, 300, 1.1, 9.9] // column: [1, b, null, 9.9, 300, 1, z, null, 1.1, b] - BitmapColumnIndex columnIndex = nullIndex.forNull(); + BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); @@ -1203,7 +1203,7 @@ public void testVariantValueSetIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // 10 rows @@ -1261,11 +1261,11 @@ public void testVariantRangeIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); - LexicographicalRangeIndex rangeIndex = indexSupplier.as(LexicographicalRangeIndex.class); + LexicographicalRangeIndexes rangeIndex = indexSupplier.as(LexicographicalRangeIndexes.class); Assert.assertNull(rangeIndex); - NumericRangeIndex numericRangeIndex = indexSupplier.as(NumericRangeIndex.class); - Assert.assertNull(numericRangeIndex); + NumericRangeIndexes numericRangeIndexes = indexSupplier.as(NumericRangeIndexes.class); + Assert.assertNull(numericRangeIndexes); } @Test @@ -1273,7 +1273,7 @@ public void testVariantPredicateIndex() throws IOException { NestedFieldColumnIndexSupplier indexSupplier = makeVariantSupplierWithNull(); - DruidPredicateIndex predicateIndex = indexSupplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = indexSupplier.as(DruidPredicateIndexes.class); Assert.assertNotNull(predicateIndex); DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( null, @@ -1433,7 +1433,7 @@ public void testEnsureNoImproperSelectionFromAdjustedGlobals() throws IOExceptio ROW_COUNT ); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // 3 rows @@ -1487,16 +1487,16 @@ public double skipValuePredicateIndexScale() null, new InDimFilter.ValuesSet(ImmutableSet.of("0")) ); - Assert.assertNull(singleTypeStringSupplier.as(DruidPredicateIndex.class).forPredicate(predicateFactory)); - Assert.assertNull(singleTypeLongSupplier.as(DruidPredicateIndex.class).forPredicate(predicateFactory)); - Assert.assertNull(singleTypeDoubleSupplier.as(DruidPredicateIndex.class).forPredicate(predicateFactory)); - Assert.assertNull(variantSupplierWithNull.as(DruidPredicateIndex.class).forPredicate(predicateFactory)); + Assert.assertNull(singleTypeStringSupplier.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); + Assert.assertNull(singleTypeLongSupplier.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); + Assert.assertNull(singleTypeDoubleSupplier.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); + Assert.assertNull(variantSupplierWithNull.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); // range index computation is a bit more complicated and done inside of the index maker gizmo because we don't know // the range up front - LexicographicalRangeIndex stringRange = singleTypeStringSupplier.as(LexicographicalRangeIndex.class); - NumericRangeIndex longRange = singleTypeLongSupplier.as(NumericRangeIndex.class); - NumericRangeIndex doubleRange = singleTypeDoubleSupplier.as(NumericRangeIndex.class); + LexicographicalRangeIndexes stringRange = singleTypeStringSupplier.as(LexicographicalRangeIndexes.class); + NumericRangeIndexes longRanges = singleTypeLongSupplier.as(NumericRangeIndexes.class); + NumericRangeIndexes doubleRanges = singleTypeDoubleSupplier.as(NumericRangeIndexes.class); // string: [b, foo, fooo, z] // small enough should be cool @@ -1508,40 +1508,40 @@ public double skipValuePredicateIndexScale() // long: [1, 3, 100, 300] // small enough should be cool - Assert.assertNotNull(longRange.forRange(1, false, 100, true)); + Assert.assertNotNull(longRanges.forRange(1, false, 100, true)); // range too big, no index - Assert.assertNull(longRange.forRange(1, false, null, false)); + Assert.assertNull(longRanges.forRange(1, false, null, false)); // double: [1.1, 1.2, 3.3, 6.6] // small enough should be cool - Assert.assertNotNull(doubleRange.forRange(null, false, 1.2, false)); + Assert.assertNotNull(doubleRanges.forRange(null, false, 1.2, false)); // range too big, no index - Assert.assertNull(doubleRange.forRange(null, false, 3.3, false)); + Assert.assertNull(doubleRanges.forRange(null, false, 3.3, false)); // other index types should not be impacted Assert.assertNotNull(singleTypeStringSupplier.as(DictionaryEncodedStringValueIndex.class)); Assert.assertNotNull(singleTypeStringSupplier.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(singleTypeStringSupplier.as(StringValueSetIndex.class).forValue("foo")); + Assert.assertNotNull(singleTypeStringSupplier.as(StringValueSetIndexes.class).forValue("foo")); Assert.assertNotNull( - singleTypeStringSupplier.as(StringValueSetIndex.class) + singleTypeStringSupplier.as(StringValueSetIndexes.class) .forSortedValues(new TreeSet<>(ImmutableSet.of("foo", "fooo", "z"))) ); Assert.assertNotNull(singleTypeStringSupplier.as(NullValueIndex.class)); Assert.assertNotNull(singleTypeLongSupplier.as(DictionaryEncodedStringValueIndex.class)); Assert.assertNotNull(singleTypeLongSupplier.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(singleTypeLongSupplier.as(StringValueSetIndex.class).forValue("1")); + Assert.assertNotNull(singleTypeLongSupplier.as(StringValueSetIndexes.class).forValue("1")); Assert.assertNotNull( - singleTypeLongSupplier.as(StringValueSetIndex.class) + singleTypeLongSupplier.as(StringValueSetIndexes.class) .forSortedValues(new TreeSet<>(ImmutableSet.of("1", "3", "100"))) ); Assert.assertNotNull(singleTypeLongSupplier.as(NullValueIndex.class)); Assert.assertNotNull(singleTypeDoubleSupplier.as(DictionaryEncodedStringValueIndex.class)); Assert.assertNotNull(singleTypeDoubleSupplier.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(singleTypeDoubleSupplier.as(StringValueSetIndex.class).forValue("1.1")); + Assert.assertNotNull(singleTypeDoubleSupplier.as(StringValueSetIndexes.class).forValue("1.1")); Assert.assertNotNull( - singleTypeDoubleSupplier.as(StringValueSetIndex.class) + singleTypeDoubleSupplier.as(StringValueSetIndexes.class) .forSortedValues(new TreeSet<>(ImmutableSet.of("1.1", "1.2", "3.3"))) ); Assert.assertNotNull(singleTypeDoubleSupplier.as(NullValueIndex.class)); @@ -1549,9 +1549,9 @@ public double skipValuePredicateIndexScale() // variant: [null, b, z, 1, 300, 1.1, 9.9] Assert.assertNotNull(variantSupplierWithNull.as(DictionaryEncodedStringValueIndex.class)); Assert.assertNotNull(variantSupplierWithNull.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(variantSupplierWithNull.as(StringValueSetIndex.class).forValue("b")); + Assert.assertNotNull(variantSupplierWithNull.as(StringValueSetIndexes.class).forValue("b")); Assert.assertNotNull( - variantSupplierWithNull.as(StringValueSetIndex.class) + variantSupplierWithNull.as(StringValueSetIndexes.class) .forSortedValues(new TreeSet<>(ImmutableSet.of("b", "1", "9.9"))) ); Assert.assertNotNull(variantSupplierWithNull.as(NullValueIndex.class)); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java index 8d1e535d182d..383a2046db9e 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -42,11 +42,11 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.vector.NoFilterVectorOffset; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -248,8 +248,8 @@ private void smokeTest(ScalarDoubleColumnAndIndexSupplier supplier, ScalarDouble ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); VectorValueSelector vectorValueSelector = column.makeVectorValueSelector(vectorOffset); - StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); - DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + StringValueSetIndexes valueSetIndex = supplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes predicateIndex = supplier.as(DruidPredicateIndexes.class); NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); SortedMap fields = column.getFieldTypeInfo(); @@ -292,7 +292,7 @@ private void smokeTest(ScalarDoubleColumnAndIndexSupplier supplier, ScalarDouble Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) .computeBitmapResult(resultFactory) .get(i)); - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); } else { if (NullHandling.sqlCompatible()) { @@ -300,7 +300,7 @@ private void smokeTest(ScalarDoubleColumnAndIndexSupplier supplier, ScalarDouble Assert.assertTrue(valueSelector.isNull()); Assert.assertTrue(vectorValueSelector.getNullVector()[0]); Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) .computeBitmapResult(resultFactory) .get(i)); @@ -311,7 +311,7 @@ private void smokeTest(ScalarDoubleColumnAndIndexSupplier supplier, ScalarDouble Assert.assertNull(vectorValueSelector.getNullVector()); Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) .computeBitmapResult(resultFactory) .get(i)); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java index 415bcf0d4020..87c4fcd8303a 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -42,11 +42,11 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.vector.NoFilterVectorOffset; import org.apache.druid.segment.vector.VectorValueSelector; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; @@ -248,8 +248,8 @@ private void smokeTest(ScalarLongColumnAndIndexSupplier supplier, ScalarLongColu ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); VectorValueSelector vectorValueSelector = column.makeVectorValueSelector(vectorOffset); - StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); - DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + StringValueSetIndexes valueSetIndex = supplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes predicateIndex = supplier.as(DruidPredicateIndexes.class); NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); SortedMap fields = column.getFieldTypeInfo(); @@ -292,7 +292,7 @@ private void smokeTest(ScalarLongColumnAndIndexSupplier supplier, ScalarLongColu Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) .computeBitmapResult(resultFactory) .get(i)); - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); } else { if (NullHandling.sqlCompatible()) { @@ -300,7 +300,7 @@ private void smokeTest(ScalarLongColumnAndIndexSupplier supplier, ScalarLongColu Assert.assertTrue(valueSelector.isNull()); Assert.assertTrue(vectorValueSelector.getNullVector()[0]); Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) .computeBitmapResult(resultFactory) .get(i)); @@ -309,7 +309,7 @@ private void smokeTest(ScalarLongColumnAndIndexSupplier supplier, ScalarLongColu Assert.assertFalse(valueSelector.isNull()); Assert.assertNull(vectorValueSelector.getNullVector()); Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) .computeBitmapResult(resultFactory) .get(i)); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index 5b86747aa2ae..da5edc734e61 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -43,12 +43,12 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory; import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory; import org.apache.druid.testing.InitializedNullHandlingTest; @@ -248,8 +248,8 @@ private void smokeTest(ScalarStringColumnAndIndexSupplier supplier, StringUtf8Di ColumnValueSelector valueSelector = column.makeColumnValueSelector(offset); DimensionSelector dimSelector = column.makeDimensionSelector(offset, null); - StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); - DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + StringValueSetIndexes valueSetIndex = supplier.as(StringValueSetIndexes.class); + DruidPredicateIndexes predicateIndex = supplier.as(DruidPredicateIndexes.class); NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); SortedMap fields = column.getFieldTypeInfo(); @@ -283,7 +283,7 @@ private void smokeTest(ScalarStringColumnAndIndexSupplier supplier, StringUtf8Di Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) .computeBitmapResult(resultFactory) .get(i)); - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); Assert.assertTrue(dimSelector.makeValueMatcher(row).matches()); Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); @@ -298,7 +298,7 @@ private void smokeTest(ScalarStringColumnAndIndexSupplier supplier, StringUtf8Di Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) .computeBitmapResult(resultFactory) .get(i)); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 9c7375ca4fd5..69fe1dc9bfed 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -42,14 +42,14 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DruidPredicateIndex; -import org.apache.druid.segment.column.NullValueIndex; import org.apache.druid.segment.column.StringEncodingStrategy; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.FrontCodedIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; +import org.apache.druid.segment.index.semantic.NullValueIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.vector.NoFilterVectorOffset; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; import org.apache.druid.segment.vector.VectorObjectSelector; @@ -371,9 +371,9 @@ private void smokeTest( SingleValueDimensionVectorSelector dimensionVectorSelector = expectedLogicalType.isPrimitive() ? column.makeSingleValueDimensionVectorSelector(vectorOffset) : null; - StringValueSetIndex valueSetIndex = supplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = supplier.as(StringValueSetIndexes.class); Assert.assertNull(valueSetIndex); - DruidPredicateIndex predicateIndex = supplier.as(DruidPredicateIndex.class); + DruidPredicateIndexes predicateIndex = supplier.as(DruidPredicateIndexes.class); Assert.assertNull(predicateIndex); NullValueIndex nullValueIndex = supplier.as(NullValueIndex.class); Assert.assertNotNull(nullValueIndex); @@ -427,7 +427,7 @@ private void smokeTest( } } } - Assert.assertFalse(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); } else { Assert.assertNull(valueSelector.getObject()); @@ -439,7 +439,7 @@ private void smokeTest( Assert.assertNull(dimensionVectorSelector.lookupName(dimensionVectorSelector.getRowVector()[0])); } } - Assert.assertTrue(nullValueIndex.forNull().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); } offset.increment(); diff --git a/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java index 7537dc53c92f..8843bd788565 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java @@ -24,12 +24,12 @@ import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.DefaultBitmapResultFactory; -import org.apache.druid.segment.column.BitmapColumnIndex; -import org.apache.druid.segment.column.StringValueSetIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.GenericIndexedWriter; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.writeout.OnHeapMemorySegmentWriteOutMedium; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -53,7 +53,7 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan public void testStringColumnWithNullValueSetIndex() throws IOException { StringUtf8ColumnIndexSupplier indexSupplier = makeStringWithNullsSupplier(); - StringValueSetIndex valueSetIndex = indexSupplier.as(StringValueSetIndex.class); + StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); Assert.assertNotNull(valueSetIndex); // 10 rows diff --git a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java index 2c3455478d0f..c6e6e9359b4a 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/NullColumnPartSerdeTest.java @@ -71,7 +71,6 @@ public void testDeserializer() Assert.assertTrue(Types.is(columnCapabilities, ValueType.DOUBLE)); Assert.assertTrue(columnCapabilities.hasNulls().isTrue()); Assert.assertTrue(columnCapabilities.hasMultipleValues().isFalse()); - Assert.assertTrue(columnCapabilities.isFilterable()); Assert.assertTrue(columnCapabilities.hasBitmapIndexes()); Assert.assertTrue(columnCapabilities.isDictionaryEncoded().isTrue()); Assert.assertTrue(columnCapabilities.areDictionaryValuesSorted().isTrue()); diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java index 4b7e1f640b20..d642acb4d69a 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumnSelectorTest.java @@ -39,10 +39,10 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.filter.SelectorFilter; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.easymock.EasyMock; import org.junit.Assert; diff --git a/processing/src/test/resources/nested-all-types-test-data.json b/processing/src/test/resources/nested-all-types-test-data.json index 832c4de7cf00..95a43a70d1cd 100644 --- a/processing/src/test/resources/nested-all-types-test-data.json +++ b/processing/src/test/resources/nested-all-types-test-data.json @@ -1,7 +1,7 @@ {"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} {"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantEmptyObj":"b", "variantEmtpyArray":2, "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}, [1, 2, 3]], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} {"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "obj":{"a": 300}, "complexObj":{"x": 4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "4", "variantEmptyObj":{}, "variantEmtpyArray":4, "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "1", "variantEmptyObj":{}, "variantEmtpyArray":4, "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} {"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantEmptyObj":{}, "variantEmtpyArray":[], "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} {"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantEmptyObj":1, "variantEmtpyArray":[], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantEmptyObj":1, "variantEmtpyArray":[], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} diff --git a/server/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java b/server/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java index d0097bcc8108..65a3e68dcfea 100644 --- a/server/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java +++ b/server/src/test/java/org/apache/druid/query/expression/LookupExprMacroTest.java @@ -34,7 +34,7 @@ public class LookupExprMacroTest extends InitializedNullHandlingTest { private static final Expr.ObjectBinding BINDINGS = InputBindings.forInputSuppliers( - ImmutableMap.builder() + ImmutableMap.>builder() .put("x", InputBindings.inputSupplier(ExpressionType.STRING, () -> "foo")) .build() ); diff --git a/services/src/main/java/org/apache/druid/cli/DumpSegment.java b/services/src/main/java/org/apache/druid/cli/DumpSegment.java index 8c865af1664a..175c4332a954 100644 --- a/services/src/main/java/org/apache/druid/cli/DumpSegment.java +++ b/services/src/main/java/org/apache/druid/cli/DumpSegment.java @@ -85,13 +85,13 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.ConciseBitmapSerdeFactory; import org.apache.druid.segment.data.FixedIndexed; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.nested.CompressedNestedDataComplexColumn; import org.apache.druid.segment.nested.NestedFieldDictionaryEncodedColumn; import org.apache.druid.segment.nested.NestedPathFinder; diff --git a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java index 6613dc212fd4..10027ae73b50 100644 --- a/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java +++ b/services/src/test/java/org/apache/druid/cli/DumpSegmentTest.java @@ -50,7 +50,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.column.DictionaryEncodedStringValueIndex; +import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.junit.After; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java index 9d5023e8baf0..d8758141dfba 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/BitwiseSqlAggregator.java @@ -32,11 +32,13 @@ import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Optionality; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -169,7 +171,11 @@ public Aggregation toDruidAggregation( null, macroTable ), - new NotDimFilter(new SelectorDimFilter(fieldName, null, null)) + new NotDimFilter( + plannerContext.isUseBoundsAndSelectors() + ? new SelectorDimFilter(fieldName, NullHandling.defaultStringValue(), null) + : NullFilter.forColumn(fieldName) + ) ) ); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java index a6044d53e364..e235cfc06cb8 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/aggregation/builtin/StringSqlAggregator.java @@ -37,12 +37,14 @@ import org.apache.calcite.sql.type.SqlTypeFamily; import org.apache.calcite.sql.type.SqlTypeName; import org.apache.calcite.util.Optionality; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.HumanReadableBytes; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; @@ -140,7 +142,11 @@ public Aggregation toDruidAggregation( } final String finalizer = StringUtils.format("if(array_length(o) == 0, null, array_to_string(o, '%s'))", separator); - final NotDimFilter dimFilter = new NotDimFilter(new SelectorDimFilter(fieldName, null, null)); + final NotDimFilter dimFilter = new NotDimFilter( + plannerContext.isUseBoundsAndSelectors() + ? new SelectorDimFilter(fieldName, NullHandling.defaultStringValue(), null) + : NullFilter.forColumn(fieldName) + ); if (aggregateCall.isDistinct()) { return Aggregation.create( // string_agg ignores nulls diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java index f8f85d836b7d..c0f32aa5744f 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/Expressions.java @@ -45,6 +45,7 @@ import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.ordering.StringComparator; @@ -55,6 +56,8 @@ import org.apache.druid.sql.calcite.filtration.BoundRefKey; import org.apache.druid.sql.calcite.filtration.Bounds; import org.apache.druid.sql.calcite.filtration.Filtration; +import org.apache.druid.sql.calcite.filtration.RangeRefKey; +import org.apache.druid.sql.calcite.filtration.Ranges; import org.apache.druid.sql.calcite.planner.Calcites; import org.apache.druid.sql.calcite.planner.ExpressionParser; import org.apache.druid.sql.calcite.planner.PlannerContext; @@ -516,22 +519,34 @@ private static DimFilter toSimpleLeafFilter( // column instead for filtering to ensure that results are correct if (druidExpression.isSimpleExtraction() && !(isOutputNumeric && !rowSignature.isNumeric(druidExpression.getDirectColumn()))) { - equalFilter = new SelectorDimFilter( - druidExpression.getSimpleExtraction().getColumn(), - NullHandling.defaultStringValue(), - druidExpression.getSimpleExtraction().getExtractionFn() - ); + if (!plannerContext.isUseBoundsAndSelectors()) { + equalFilter = new NullFilter( + druidExpression.getSimpleExtraction().getColumn(), + druidExpression.getSimpleExtraction().getExtractionFn(), + null + ); + } else { + equalFilter = new SelectorDimFilter( + druidExpression.getSimpleExtraction().getColumn(), + NullHandling.defaultStringValue(), + druidExpression.getSimpleExtraction().getExtractionFn() + ); + } } else if (virtualColumnRegistry != null) { final String virtualColumn = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( druidExpression, operand.getType() ); - equalFilter = new SelectorDimFilter( - virtualColumn, - NullHandling.defaultStringValue(), - null - ); + if (!plannerContext.isUseBoundsAndSelectors()) { + equalFilter = new NullFilter(virtualColumn, null, null); + } else { + equalFilter = new SelectorDimFilter( + virtualColumn, + NullHandling.defaultStringValue(), + null + ); + } } else { return null; } @@ -585,8 +600,12 @@ private static DimFilter toSimpleLeafFilter( flippedKind = kind; } + final DruidExpression rhsExpression = toDruidExpression(plannerContext, rowSignature, rhs); + final Expr parsedRhsExpression = rhsExpression != null + ? plannerContext.parseExpression(rhsExpression.getExpression()) + : null; // rhs must be a literal - if (rhs.getKind() != SqlKind.LITERAL) { + if (rhs.getKind() != SqlKind.LITERAL && (parsedRhsExpression == null || !parsedRhsExpression.isLiteral())) { return null; } @@ -602,7 +621,13 @@ private static DimFilter toSimpleLeafFilter( if (queryGranularity != null) { // lhs is FLOOR(__time TO granularity); rhs must be a timestamp final long rhsMillis = Calcites.calciteDateTimeLiteralToJoda(rhs, plannerContext.getTimeZone()).getMillis(); - return buildTimeFloorFilter(ColumnHolder.TIME_COLUMN_NAME, queryGranularity, flippedKind, rhsMillis); + return buildTimeFloorFilter( + ColumnHolder.TIME_COLUMN_NAME, + queryGranularity, + flippedKind, + rhsMillis, + plannerContext + ); } final String column; @@ -633,61 +658,105 @@ private static DimFilter toSimpleLeafFilter( // Is rhs aligned on granularity boundaries? final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis; - // Create a BoundRefKey that strips the extractionFn and compares __time as a number. - final BoundRefKey boundRefKey = new BoundRefKey(column, null, StringComparators.NUMERIC); + if (plannerContext.isUseBoundsAndSelectors()) { + // Create a BoundRefKey that strips the extractionFn and compares __time as a number. + final BoundRefKey boundRefKey = new BoundRefKey(column, null, StringComparators.NUMERIC); - return getBoundTimeDimFilter(flippedKind, boundRefKey, rhsInterval, rhsAligned); + return getBoundTimeDimFilter(flippedKind, boundRefKey, rhsInterval, rhsAligned); + } else { + final RangeRefKey rangeRefKey = new RangeRefKey(column, ColumnType.LONG, null); + return getRangeTimeDimFilter(flippedKind, rangeRefKey, rhsInterval, rhsAligned); + } } } - final String val; - final RexLiteral rhsLiteral = (RexLiteral) rhs; - if (SqlTypeName.NUMERIC_TYPES.contains(rhsLiteral.getTypeName())) { - val = String.valueOf(RexLiteral.value(rhsLiteral)); - } else if (SqlTypeName.CHAR_TYPES.contains(rhsLiteral.getTypeName())) { - val = String.valueOf(RexLiteral.stringValue(rhsLiteral)); - } else if (SqlTypeName.TIMESTAMP == rhsLiteral.getTypeName() || SqlTypeName.DATE == rhsLiteral.getTypeName()) { - val = String.valueOf( - Calcites.calciteDateTimeLiteralToJoda( - rhsLiteral, - plannerContext.getTimeZone() - ).getMillis() - ); + if (plannerContext.isUseBoundsAndSelectors() && rhs instanceof RexLiteral) { + final String val; + final RexLiteral rhsLiteral = (RexLiteral) rhs; + if (SqlTypeName.NUMERIC_TYPES.contains(rhsLiteral.getTypeName())) { + val = String.valueOf(RexLiteral.value(rhsLiteral)); + } else if (SqlTypeName.CHAR_TYPES.contains(rhsLiteral.getTypeName())) { + val = String.valueOf(RexLiteral.stringValue(rhsLiteral)); + } else if (SqlTypeName.TIMESTAMP == rhsLiteral.getTypeName() || SqlTypeName.DATE == rhsLiteral.getTypeName()) { + val = String.valueOf( + Calcites.calciteDateTimeLiteralToJoda( + rhsLiteral, + plannerContext.getTimeZone() + ).getMillis() + ); + } else { + // Don't know how to filter on this kind of literal. + return null; + } + // Numeric lhs needs a numeric comparison. + final StringComparator comparator = Calcites.getStringComparatorForRelDataType(lhs.getType()); + final BoundRefKey boundRefKey = new BoundRefKey(column, extractionFn, comparator); + final DimFilter filter; + + // Always use BoundDimFilters, to simplify filter optimization later (it helps to remember the comparator). + switch (flippedKind) { + case EQUALS: + filter = Bounds.equalTo(boundRefKey, val); + break; + case NOT_EQUALS: + filter = new NotDimFilter(Bounds.equalTo(boundRefKey, val)); + break; + case GREATER_THAN: + filter = Bounds.greaterThan(boundRefKey, val); + break; + case GREATER_THAN_OR_EQUAL: + filter = Bounds.greaterThanOrEqualTo(boundRefKey, val); + break; + case LESS_THAN: + filter = Bounds.lessThan(boundRefKey, val); + break; + case LESS_THAN_OR_EQUAL: + filter = Bounds.lessThanOrEqualTo(boundRefKey, val); + break; + default: + throw new IllegalStateException("Shouldn't have got here"); + } + + return filter; } else { - // Don't know how to filter on this kind of literal. - return null; - } + final Object val; + if (parsedRhsExpression != null && parsedRhsExpression.isLiteral()) { + val = parsedRhsExpression.getLiteralValue(); + } else { + // Don't know how to filter on this kind of literal. + return null; + } - // Numeric lhs needs a numeric comparison. - final StringComparator comparator = Calcites.getStringComparatorForRelDataType(lhs.getType()); - final BoundRefKey boundRefKey = new BoundRefKey(column, extractionFn, comparator); - final DimFilter filter; - - // Always use BoundDimFilters, to simplify filter optimization later (it helps to remember the comparator). - switch (flippedKind) { - case EQUALS: - filter = Bounds.equalTo(boundRefKey, val); - break; - case NOT_EQUALS: - filter = new NotDimFilter(Bounds.equalTo(boundRefKey, val)); - break; - case GREATER_THAN: - filter = Bounds.greaterThan(boundRefKey, val); - break; - case GREATER_THAN_OR_EQUAL: - filter = Bounds.greaterThanOrEqualTo(boundRefKey, val); - break; - case LESS_THAN: - filter = Bounds.lessThan(boundRefKey, val); - break; - case LESS_THAN_OR_EQUAL: - filter = Bounds.lessThanOrEqualTo(boundRefKey, val); - break; - default: - throw new IllegalStateException("Shouldn't have got here"); - } + final ColumnType matchValueType = Calcites.getColumnTypeForRelDataType(rhs.getType()); + final RangeRefKey rangeRefKey = new RangeRefKey(column, matchValueType, extractionFn); + final DimFilter filter; + + // Always use RangeFilter, to simplify filter optimization later + switch (flippedKind) { + case EQUALS: + filter = Ranges.equalTo(rangeRefKey, val); + break; + case NOT_EQUALS: + filter = new NotDimFilter(Ranges.equalTo(rangeRefKey, val)); + break; + case GREATER_THAN: + filter = Ranges.greaterThan(rangeRefKey, val); + break; + case GREATER_THAN_OR_EQUAL: + filter = Ranges.greaterThanOrEqualTo(rangeRefKey, val); + break; + case LESS_THAN: + filter = Ranges.lessThan(rangeRefKey, val); + break; + case LESS_THAN_OR_EQUAL: + filter = Ranges.lessThanOrEqualTo(rangeRefKey, val); + break; + default: + throw new IllegalStateException("Shouldn't have got here"); + } - return filter; + return filter; + } } else if (rexNode instanceof RexCall) { final SqlOperator operator = ((RexCall) rexNode).getOperator(); final SqlOperatorConversion conversion = plannerContext.getPlannerToolbox().operatorTable().lookupOperatorConversion(operator); @@ -772,16 +841,27 @@ private static DimFilter buildTimeFloorFilter( final String column, final Granularity granularity, final SqlKind operatorKind, - final long rhsMillis + final long rhsMillis, + final PlannerContext plannerContext ) { - final BoundRefKey boundRefKey = new BoundRefKey(column, null, StringComparators.NUMERIC); - final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis)); + if (plannerContext.isUseBoundsAndSelectors()) { + final BoundRefKey boundRefKey = new BoundRefKey(column, null, StringComparators.NUMERIC); + final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis)); - // Is rhs aligned on granularity boundaries? - final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis; + // Is rhs aligned on granularity boundaries? + final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis; - return getBoundTimeDimFilter(operatorKind, boundRefKey, rhsInterval, rhsAligned); + return getBoundTimeDimFilter(operatorKind, boundRefKey, rhsInterval, rhsAligned); + } else { + final RangeRefKey rangeRefKey = new RangeRefKey(column, ColumnType.LONG, null); + final Interval rhsInterval = granularity.bucket(DateTimes.utc(rhsMillis)); + + // Is rhs aligned on granularity boundaries? + final boolean rhsAligned = rhsInterval.getStartMillis() == rhsMillis; + + return getRangeTimeDimFilter(operatorKind, rangeRefKey, rhsInterval, rhsAligned); + } } @@ -817,4 +897,37 @@ private static DimFilter getBoundTimeDimFilter( throw new IllegalStateException("Shouldn't have got here"); } } + + private static DimFilter getRangeTimeDimFilter( + SqlKind operatorKind, + RangeRefKey rangeRefKey, + Interval interval, + boolean isAligned + ) + { + switch (operatorKind) { + case EQUALS: + return isAligned + ? Ranges.interval(rangeRefKey, interval) + : Filtration.matchNothing(); + case NOT_EQUALS: + return isAligned + ? new NotDimFilter(Ranges.interval(rangeRefKey, interval)) + : Filtration.matchEverything(); + case GREATER_THAN: + return Ranges.greaterThanOrEqualTo(rangeRefKey, String.valueOf(interval.getEndMillis())); + case GREATER_THAN_OR_EQUAL: + return isAligned + ? Ranges.greaterThanOrEqualTo(rangeRefKey, String.valueOf(interval.getStartMillis())) + : Ranges.greaterThanOrEqualTo(rangeRefKey, String.valueOf(interval.getEndMillis())); + case LESS_THAN: + return isAligned + ? Ranges.lessThan(rangeRefKey, String.valueOf(interval.getStartMillis())) + : Ranges.lessThan(rangeRefKey, String.valueOf(interval.getEndMillis())); + case LESS_THAN_OR_EQUAL: + return Ranges.lessThan(rangeRefKey, String.valueOf(interval.getEndMillis())); + default: + throw new IllegalStateException("Shouldn't have got here"); + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java index 36c5fd77d099..b53a96d45716 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayContainsOperatorConversion.java @@ -28,9 +28,11 @@ import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.InputBindings; import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.expression.DruidExpression; import org.apache.druid.sql.calcite.expression.Expressions; @@ -94,6 +96,8 @@ public DimFilter toDruidFilter( final DruidExpression leftExpr = druidExpressions.get(0); final DruidExpression rightExpr = druidExpressions.get(1); + // if the input column is not actually an ARRAY type, but rather an MVD, we can optimize this into + // selector/equality filters on the individual array elements if (leftExpr.isSimpleExtraction() && !(leftExpr.isDirectColumnAccess() && leftExpr.getDruidType() != null && leftExpr.getDruidType().isArray())) { Expr expr = plannerContext.parseExpression(rightExpr.getExpression()); // To convert this expression filter into an And of Selector filters, we need to extract all array elements. @@ -112,11 +116,33 @@ public DimFilter toDruidFilter( // to create an empty array with no argument, we just return null. return null; } else if (arrayElements.length == 1) { - return newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(arrayElements[0])); + if (plannerContext.isUseBoundsAndSelectors()) { + return newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(arrayElements[0])); + } else { + return new EqualityFilter( + leftExpr.getSimpleExtraction().getColumn(), + ExpressionType.toColumnType(ExpressionType.elementType(exprEval.type())), + arrayElements[0], + leftExpr.getSimpleExtraction().getExtractionFn(), + null + ); + } } else { final List selectFilters = Arrays .stream(arrayElements) - .map(val -> newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(val))) + .map(val -> { + if (plannerContext.isUseBoundsAndSelectors()) { + return newSelectorDimFilter(leftExpr.getSimpleExtraction(), Evals.asString(val)); + } else { + return new EqualityFilter( + leftExpr.getSimpleExtraction().getColumn(), + ExpressionType.toColumnType(ExpressionType.elementType(exprEval.type())), + val, + leftExpr.getSimpleExtraction().getExtractionFn(), + null + ); + } + }) .collect(Collectors.toList()); return new AndDimFilter(selectFilters); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java index b0e25e14737a..68a3d9ff7e93 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/ArrayOverlapOperatorConversion.java @@ -28,8 +28,10 @@ import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.InputBindings; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -124,7 +126,17 @@ public DimFilter toDruidFilter( // to create an empty array with no argument, we just return null. return null; } else if (arrayElements.length == 1) { - return newSelectorDimFilter(simpleExtractionExpr.getSimpleExtraction(), Evals.asString(arrayElements[0])); + if (plannerContext.isUseBoundsAndSelectors()) { + return newSelectorDimFilter(simpleExtractionExpr.getSimpleExtraction(), Evals.asString(arrayElements[0])); + } else { + return new EqualityFilter( + simpleExtractionExpr.getSimpleExtraction().getColumn(), + ExpressionType.toColumnType(exprEval.type()), + arrayElements[0], + simpleExtractionExpr.getSimpleExtraction().getExtractionFn(), + null + ); + } } else { return new InDimFilter( simpleExtractionExpr.getSimpleExtraction().getColumn(), diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java index 39d5c952d924..beee936b1947 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/expression/builtin/CaseOperatorConversion.java @@ -29,6 +29,7 @@ import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.ExpressionDimFilter; +import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.RowSignature; @@ -116,11 +117,20 @@ public DimFilter toDruidFilter( if (condition.getExpression().startsWith("notnull") && condition.getArguments().get(0).isDirectColumnAccess()) { DimFilter thenFilter = null, elseFilter = null; - final DimFilter isNull = new SelectorDimFilter( - condition.getArguments().get(0).getDirectColumn(), - null, - null - ); + final DimFilter isNull; + if (plannerContext.isUseBoundsAndSelectors()) { + isNull = new SelectorDimFilter( + condition.getArguments().get(0).getDirectColumn(), + null, + null + ); + } else { + isNull = new NullFilter( + condition.getArguments().get(0).getDirectColumn(), + null, + null + ); + } if (call.getOperands().get(1) instanceof RexCall) { final RexCall thenCall = (RexCall) call.getOperands().get(1); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java index 7b4f4b669ec9..a11ffd64ed8a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CombineAndSimplifyBounds.java @@ -21,6 +21,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import org.apache.druid.java.util.common.ISE; @@ -30,6 +31,8 @@ import org.apache.druid.query.filter.FalseDimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.RangeFilter; +import org.apache.druid.segment.column.ColumnType; import java.util.ArrayList; import java.util.HashMap; @@ -133,6 +136,9 @@ private static DimFilter doSimplify(final List children, boolean disj // Group Bound filters by dimension, extractionFn, and comparator and compute a RangeSet for each one. final Map> bounds = new HashMap<>(); + // Group range filters by dimension, extractionFn, and matchValueType and compute a RangeSet for each one. + final Map> ranges = new HashMap<>(); + final Map leastRestrictiveNumericTypes = new HashMap<>(); // all and/or filters have at least 1 child boolean allFalse = true; @@ -143,8 +149,20 @@ private static DimFilter doSimplify(final List children, boolean disj final List filterList = bounds.computeIfAbsent(boundRefKey, k -> new ArrayList<>()); filterList.add(bound); allFalse = false; + } else if (child instanceof RangeFilter) { + final RangeFilter range = (RangeFilter) child; + final RangeRefKey rangeRefKey = RangeRefKey.from(range); + if (rangeRefKey.getMatchValueType().isNumeric()) { + leastRestrictiveNumericTypes.compute( + range.getColumn(), + (c, existingType) -> ColumnType.leastRestrictiveType(existingType, range.getMatchValueType()) + ); + } + final List filterList = ranges.computeIfAbsent(rangeRefKey, k -> new ArrayList<>()); + filterList.add(range); + allFalse = false; } else { - allFalse &= child instanceof FalseDimFilter; + allFalse = allFalse && (child instanceof FalseDimFilter); } } @@ -188,6 +206,63 @@ private static DimFilter doSimplify(final List children, boolean disj } } + // Try to consolidate numeric groups + final Map> consolidatedNumericRanges = Maps.newHashMapWithExpectedSize(ranges.size()); + for (Map.Entry> entry : ranges.entrySet()) { + RangeRefKey refKey = entry.getKey(); + if (entry.getKey().getMatchValueType().isNumeric()) { + ColumnType numericTypeToUse = leastRestrictiveNumericTypes.get(refKey.getColumn()); + refKey = new RangeRefKey(refKey.getColumn(), numericTypeToUse, refKey.getExtractionFn()); + } + final List filterList = consolidatedNumericRanges.computeIfAbsent(refKey, k -> new ArrayList<>()); + for (RangeFilter filter : entry.getValue()) { + + int pos = newChildren.indexOf(filter); + if (!newChildren.remove(filter)) { + // Don't expect this to happen, but include it as a sanity check. + throw new ISE("Tried to remove range, but couldn't"); + } + final RangeFilter rewrite = Ranges.toFilter(refKey, Ranges.toRange(filter, refKey.getMatchValueType())); + newChildren.add(pos, rewrite); + filterList.add(rewrite); + } + } + + // Try to simplify filters within each group. + for (Map.Entry> entry : consolidatedNumericRanges.entrySet()) { + final RangeRefKey rangeRefKey = entry.getKey(); + final List filterList = entry.getValue(); + + // Create a RangeSet for this group. + final RangeSet rangeSet = disjunction + ? RangeSets.unionRanges(Ranges.toRanges(filterList)) + : RangeSets.intersectRanges(Ranges.toRanges(filterList)); + + if (rangeSet.asRanges().size() < filterList.size()) { + // We found a simplification. Remove the old filters and add new ones. + for (final RangeFilter range : filterList) { + if (!newChildren.remove(range)) { + // Don't expect this to happen, but include it as a sanity check. + throw new ISE("Tried to remove range, but couldn't"); + } + } + + if (rangeSet.asRanges().isEmpty()) { + // range set matches nothing, equivalent to FALSE + newChildren.add(Filtration.matchNothing()); + } + + for (final Range range : rangeSet.asRanges()) { + if (!range.hasLowerBound() && !range.hasUpperBound()) { + // range matches all, equivalent to TRUE + newChildren.add(Filtration.matchEverything()); + } else { + newChildren.add(Ranges.toFilter(rangeRefKey, range)); + } + } + } + } + // Finally: Go through newChildren, removing or potentially exiting early based on TRUE / FALSE marker filters. Preconditions.checkState(newChildren.size() > 0, "newChildren.size > 0"); @@ -243,6 +318,9 @@ private static DimFilter negate(final DimFilter filter) } else if (filter instanceof BoundDimFilter) { final BoundDimFilter negated = Bounds.not((BoundDimFilter) filter); return negated != null ? negated : new NotDimFilter(filter); + } else if (filter instanceof RangeFilter) { + final RangeFilter negated = Ranges.not((RangeFilter) filter); + return negated != null ? negated : new NotDimFilter(filter); } else { return new NotDimFilter(filter); } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertBoundsToSelectors.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertBoundsToSelectors.java index 155e7a632ccf..6c5109693beb 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertBoundsToSelectors.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertBoundsToSelectors.java @@ -21,6 +21,8 @@ import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.query.filter.RangeFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.ordering.StringComparator; import org.apache.druid.segment.column.RowSignature; @@ -65,6 +67,21 @@ public DimFilter process(DimFilter filter) } else { return filter; } + } else if (filter instanceof RangeFilter) { + final RangeFilter bound = (RangeFilter) filter; + // since the range filter retains the match value type, we don't need to restrict to ranges + // that match the underlying column type + if (bound.isEquality()) { + return new EqualityFilter( + bound.getColumn(), + bound.getMatchValueType(), + bound.getUpper(), + bound.getExtractionFn(), + null + ); + } else { + return filter; + } } else { return filter; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertSelectorsToIns.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertSelectorsToIns.java index 29ea37129897..16769892443e 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertSelectorsToIns.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/ConvertSelectorsToIns.java @@ -21,11 +21,15 @@ import com.google.common.collect.Lists; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.sql.calcite.expression.SimpleExtraction; import org.apache.druid.sql.calcite.table.RowSignatures; @@ -57,6 +61,7 @@ public DimFilter process(DimFilter filter) // Group filters by dimension and extractionFn. final Map> selectors = new HashMap<>(); + final Map> equality = new HashMap<>(); for (DimFilter child : children) { if (child instanceof SelectorDimFilter) { @@ -70,6 +75,16 @@ public DimFilter process(DimFilter filter) ); List filterList = selectors.computeIfAbsent(boundRefKey, k -> new ArrayList<>()); filterList.add(selector); + } else if (child instanceof EqualityFilter) { + final EqualityFilter equals = (EqualityFilter) child; + if (!equals.getMatchValueType().is(ValueType.STRING)) { + // skip non-string equality filters since InDimFilter uses a sorted string set, which is a different sort + // than numbers or other types might use + continue; + } + final RangeRefKey rangeRefKey = RangeRefKey.from(equals); + List filterList = equality.computeIfAbsent(rangeRefKey, k -> new ArrayList<>()); + filterList.add(equals); } } @@ -92,6 +107,33 @@ public DimFilter process(DimFilter filter) } } + // Emit IN filters for each group of size > 1 + // right now we only do this for string types, since the value set is sorted in string order + // someday we might want to either allow numbers after ensuring that all value set indexes can handle value + // sets which are not in the correct sorted order, or make a cooler in filter that retains the match value type + // and can sort the values in match value native order + for (Map.Entry> entry : equality.entrySet()) { + final List filterList = entry.getValue(); + if (filterList.size() > 1) { + // We found a simplification. Remove the old filters and add new ones. + final InDimFilter.ValuesSet values = new InDimFilter.ValuesSet(); + + for (final EqualityFilter equals : filterList) { + values.add( + ExprEval.ofType(ExpressionType.fromColumnType(equals.getMatchValueType()), equals.getMatchValue()) + .castTo(ExpressionType.STRING) + .asString() + ); + if (!children.remove(equals)) { + // Don't expect this to happen, but include it as a sanity check. + throw new ISE("Tried to remove equals but couldn't"); + } + } + + children.add(new InDimFilter(entry.getKey().getColumn(), values, entry.getKey().getExtractionFn(), null)); + } + } + if (!children.equals(((OrDimFilter) filter).getFields())) { return children.size() == 1 ? children.get(0) : new OrDimFilter(children); } else { diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/MoveTimeFiltersToIntervals.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/MoveTimeFiltersToIntervals.java index b374d2523744..dc93dc1bb20c 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/MoveTimeFiltersToIntervals.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/MoveTimeFiltersToIntervals.java @@ -23,14 +23,17 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Range; import com.google.common.collect.RangeSet; +import org.apache.druid.java.util.common.Numbers; import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.RangeFilter; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.segment.column.ColumnHolder; +import org.apache.druid.segment.column.ColumnType; import java.util.ArrayList; import java.util.List; @@ -44,6 +47,12 @@ public class MoveTimeFiltersToIntervals implements Function> extractConvertibleTimeBounds(fina } else { return Pair.of(filter, null); } + } else if (filter instanceof RangeFilter) { + final RangeFilter bound = (RangeFilter) filter; + if (RangeRefKey.from(bound).equals(TIME_RANGE_REF_KEY)) { + return Pair.of(null, RangeSets.of(toLongRangeFromRange(Ranges.toRange(bound)))); + } else { + return Pair.of(filter, null); + } } else { return Pair.of(filter, null); } @@ -169,4 +185,20 @@ private static Range toLongRange(final Range range) ); } } + + private static Range toLongRangeFromRange(final Range range) + { + if (!range.hasUpperBound() && !range.hasLowerBound()) { + return Range.all(); + } else if (range.hasUpperBound() && !range.hasLowerBound()) { + return Range.upTo(Numbers.parseLong(range.upperEndpoint().getValue()), range.upperBoundType()); + } else if (!range.hasUpperBound() && range.hasLowerBound()) { + return Range.downTo(Numbers.parseLong(range.lowerEndpoint().getValue()), range.lowerBoundType()); + } else { + return Range.range( + Numbers.parseLong(range.lowerEndpoint().getValue()), range.lowerBoundType(), + Numbers.parseLong(range.upperEndpoint().getValue()), range.upperBoundType() + ); + } + } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RangeRefKey.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RangeRefKey.java new file mode 100644 index 000000000000..0547e87865f9 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RangeRefKey.java @@ -0,0 +1,107 @@ +/* + * 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.filtration; + +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.query.filter.RangeFilter; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class RangeRefKey +{ + private final String column; + private final ColumnType matchValueType; + @Nullable + private final ExtractionFn extractionFn; + + public RangeRefKey(String column, ColumnType matchValueType, ExtractionFn extractionFn) + { + this.column = column; + this.matchValueType = matchValueType; + this.extractionFn = extractionFn; + } + + public static RangeRefKey from(RangeFilter filter) + { + return new RangeRefKey( + filter.getColumn(), + filter.getMatchValueType(), + filter.getExtractionFn() + ); + } + + public static RangeRefKey from(EqualityFilter filter) + { + return new RangeRefKey( + filter.getColumn(), + filter.getMatchValueType(), + filter.getExtractionFn() + ); + } + + public String getColumn() + { + return column; + } + + public ColumnType getMatchValueType() + { + return matchValueType; + } + + public ExtractionFn getExtractionFn() + { + return extractionFn; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RangeRefKey that = (RangeRefKey) o; + return Objects.equals(column, that.column) + && Objects.equals(matchValueType, that.matchValueType) + && Objects.equals(extractionFn, that.extractionFn); + } + + @Override + public int hashCode() + { + return Objects.hash(column, matchValueType, extractionFn); + } + + @Override + public String toString() + { + return "RangeRefKey{" + + "column='" + column + '\'' + + ", matchValueType=" + matchValueType + + ", extractionFn=" + extractionFn + + '}'; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RangeValue.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RangeValue.java new file mode 100644 index 000000000000..2cccdfe9ca5a --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/RangeValue.java @@ -0,0 +1,93 @@ +/* + * 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.filtration; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; +import java.util.Comparator; +import java.util.Objects; + +public class RangeValue implements Comparable +{ + @Nullable + private final Object value; + private final ColumnType matchValueType; + private final Comparator matchValueTypeComparator; + + public RangeValue( + @Nullable Object value, + ColumnType matchValueType + ) + { + this.value = value; + this.matchValueType = matchValueType; + this.matchValueTypeComparator = matchValueType.getNullableStrategy(); + } + + @Nullable + public Object getValue() + { + return value; + } + + public ColumnType getMatchValueType() + { + return matchValueType; + } + + @Override + public int compareTo(RangeValue o) + { + if (!matchValueType.equals(o.matchValueType)) { + throw new ISE("Comparator mismatch: [%s] and [%s]", matchValueType, o.matchValueType); + } + return matchValueTypeComparator.compare(value, o.value); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RangeValue that = (RangeValue) o; + return Objects.equals(value, that.value) && Objects.equals(matchValueType, that.matchValueType); + } + + @Override + public int hashCode() + { + return Objects.hash(value, matchValueType); + } + + @Override + public String toString() + { + return "RangeValue{" + + "value=" + value + + ", matchValueType=" + matchValueType + + '}'; + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/Ranges.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/Ranges.java new file mode 100644 index 000000000000..2165e8bc7dc1 --- /dev/null +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/Ranges.java @@ -0,0 +1,225 @@ +/* + * 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.filtration; + +import com.google.common.collect.BoundType; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.math.expr.ExprEval; +import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.filter.RangeFilter; +import org.apache.druid.segment.column.ColumnType; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.List; + +public class Ranges +{ + /** + * Negates single-ended Bound filters. + * + * @param range filter + * @return negated filter, or null if this range is double-ended. + */ + @Nullable + public static RangeFilter not(final RangeFilter range) + { + if (range.getUpper() != null && range.getLower() != null) { + return null; + } else if (range.getUpper() != null) { + return new RangeFilter( + range.getColumn(), + range.getMatchValueType(), + range.getUpper(), + null, + !range.isUpperOpen(), + false, + range.getExtractionFn(), + range.getFilterTuning() + ); + } else { + // range.getLower() != null + return new RangeFilter( + range.getColumn(), + range.getMatchValueType(), + null, + range.getLower(), + false, + !range.isLowerOpen(), + range.getExtractionFn(), + range.getFilterTuning() + ); + } + } + + public static Range toRange(final RangeFilter range) + { + final RangeValue upper = range.getUpper() != null + ? new RangeValue(range.getUpper(), range.getMatchValueType()) + : null; + final RangeValue lower = range.getLower() != null + ? new RangeValue(range.getLower(), range.getMatchValueType()) + : null; + + if (lower == null) { + return range.isUpperOpen() ? Range.lessThan(upper) : Range.atMost(upper); + } else if (upper == null) { + return range.isLowerOpen() ? Range.greaterThan(lower) : Range.atLeast(lower); + } else { + BoundType lowerBoundType = range.isLowerOpen() ? BoundType.OPEN : BoundType.CLOSED; + BoundType upperBoundType = range.isUpperOpen() ? BoundType.OPEN : BoundType.CLOSED; + return Range.range(lower, lowerBoundType, upper, upperBoundType); + } + } + + public static Range toRange(final RangeFilter range, final ColumnType newMatchValueType) + { + final ExpressionType exprType = ExpressionType.fromColumnType(newMatchValueType); + final RangeValue upper = range.getUpper() != null + ? new RangeValue(ExprEval.ofType(exprType, range.getUpper()) + .valueOrDefault(), newMatchValueType) + : null; + final RangeValue lower = range.getLower() != null + ? new RangeValue(ExprEval.ofType(exprType, range.getLower()) + .valueOrDefault(), newMatchValueType) + : null; + + if (lower == null) { + return range.isUpperOpen() ? Range.lessThan(upper) : Range.atMost(upper); + } else if (upper == null) { + return range.isLowerOpen() ? Range.greaterThan(lower) : Range.atLeast(lower); + } else { + BoundType lowerBoundType = range.isLowerOpen() ? BoundType.OPEN : BoundType.CLOSED; + BoundType upperBoundType = range.isUpperOpen() ? BoundType.OPEN : BoundType.CLOSED; + return Range.range(lower, lowerBoundType, upper, upperBoundType); + } + } + + public static List> toRanges(final List ranges) + { + return ImmutableList.copyOf(Lists.transform(ranges, Ranges::toRange)); + } + + public static RangeFilter toFilter(final RangeRefKey rangeRefKey, final Range range) + { + return new RangeFilter( + rangeRefKey.getColumn(), + rangeRefKey.getMatchValueType(), + range.hasLowerBound() ? range.lowerEndpoint().getValue() : null, + range.hasUpperBound() ? range.upperEndpoint().getValue() : null, + range.hasLowerBound() && range.lowerBoundType() == BoundType.OPEN, + range.hasUpperBound() && range.upperBoundType() == BoundType.OPEN, + rangeRefKey.getExtractionFn(), + null + ); + } + + public static RangeFilter equalTo(final RangeRefKey rangeRefKey, final Object value) + { + return new RangeFilter( + rangeRefKey.getColumn(), + rangeRefKey.getMatchValueType(), + value, + value, + false, + false, + rangeRefKey.getExtractionFn(), + null + ); + } + + public static RangeFilter greaterThan(final RangeRefKey rangeRefKey, final Object value) + { + return new RangeFilter( + rangeRefKey.getColumn(), + rangeRefKey.getMatchValueType(), + value, + null, + true, + false, + rangeRefKey.getExtractionFn(), + null + ); + } + + public static RangeFilter greaterThanOrEqualTo(final RangeRefKey rangeRefKey, final Object value) + { + return new RangeFilter( + rangeRefKey.getColumn(), + rangeRefKey.getMatchValueType(), + value, + null, + false, + false, + rangeRefKey.getExtractionFn(), + null + ); + } + + public static RangeFilter lessThan(final RangeRefKey rangeRefKey, final Object value) + { + return new RangeFilter( + rangeRefKey.getColumn(), + rangeRefKey.getMatchValueType(), + null, + value, + false, + true, + rangeRefKey.getExtractionFn(), + null + ); + } + + public static RangeFilter lessThanOrEqualTo(final RangeRefKey rangeRefKey, final Object value) + { + return new RangeFilter( + rangeRefKey.getColumn(), + rangeRefKey.getMatchValueType(), + null, + value, + false, + false, + rangeRefKey.getExtractionFn(), + null + ); + } + + public static RangeFilter interval(final RangeRefKey rangeRefKey, final Interval interval) + { + if (!rangeRefKey.getMatchValueType().equals(ColumnType.LONG)) { + // Interval comparison only works with LONG comparator. + throw new ISE("Comparator must be LONG but was[%s]", rangeRefKey.getMatchValueType()); + } + + return new RangeFilter( + rangeRefKey.getColumn(), + rangeRefKey.getMatchValueType(), + interval.getStartMillis(), + interval.getEndMillis(), + false, + true, + rangeRefKey.getExtractionFn(), + null + ); + } +} diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java index 331a61a1f50b..7c066e663c44 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/Calcites.java @@ -263,9 +263,12 @@ public static RelDataType createSqlArrayTypeWithNullability( final boolean nullable ) { - final RelDataType dataType = typeFactory.createArrayType( - createSqlTypeWithNullability(typeFactory, elementTypeName, nullable), - -1 + final RelDataType dataType = typeFactory.createTypeWithNullability( + typeFactory.createArrayType( + createSqlTypeWithNullability(typeFactory, elementTypeName, nullable), + -1 + ), + true ); return dataType; 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 844d9896ae8d..c8b5c2b60dd2 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 @@ -28,6 +28,7 @@ import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExprType; +import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.InputBindings; import org.apache.druid.math.expr.Parser; import org.apache.druid.segment.column.RowSignature; @@ -145,7 +146,7 @@ public void reduce( } else if (SqlTypeName.NUMERIC_TYPES.contains(constExp.getType().getComponentType().getSqlTypeName())) { if (exprResult.type().getElementType().is(ExprType.LONG)) { List resultAsBigDecimalList = new ArrayList<>(array.length); - for (Object val : array) { + for (Object val : exprResult.castTo(ExpressionType.LONG_ARRAY).asArray()) { final Number longVal = (Number) val; if (longVal == null) { resultAsBigDecimalList.add(null); @@ -156,7 +157,7 @@ public void reduce( literal = rexBuilder.makeLiteral(resultAsBigDecimalList, constExp.getType(), true); } else { List resultAsBigDecimalList = new ArrayList<>(array.length); - for (Object val : array) { + for (Object val : exprResult.castTo(ExpressionType.fromColumnType(druidExpression.getDruidType())).asArray()) { final Number doubleVal = (Number) val; if (doubleVal == null) { resultAsBigDecimalList.add(null); diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java index abf837bb9c57..d0230da06b17 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/PlannerContext.java @@ -28,6 +28,7 @@ import org.apache.calcite.avatica.remote.TypedValue; import org.apache.calcite.linq4j.QueryProvider; import org.apache.calcite.schema.SchemaPlus; +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.Numbers; @@ -85,6 +86,10 @@ public class PlannerContext */ public static final String CTX_ENABLE_UNNEST = "enableUnnest"; + public static final String CTX_SQL_USE_BOUNDS_AND_SELECTORS = "sqlUseBoundAndSelectors"; + public static final boolean DEFAULT_SQL_USE_BOUNDS_AND_SELECTORS = NullHandling.replaceWithDefault(); + + // DataContext keys public static final String DATA_CTX_AUTHENTICATION_RESULT = "authenticationResult"; @@ -97,6 +102,7 @@ public class PlannerContext private final Map queryContext; private final String sqlQueryId; private final boolean stringifyArrays; + private final boolean useBoundsAndSelectors; private final CopyOnWriteArrayList nativeQueryIds = new CopyOnWriteArrayList<>(); private final PlannerHook hook; // bindings for dynamic parameters to bind during planning @@ -121,6 +127,7 @@ private PlannerContext( final PlannerConfig plannerConfig, final DateTime localNow, final boolean stringifyArrays, + final boolean useBoundsAndSelectors, final SqlEngine engine, final Map queryContext, final PlannerHook hook @@ -134,6 +141,7 @@ private PlannerContext( this.queryContext = queryContext; this.localNow = Preconditions.checkNotNull(localNow, "localNow"); this.stringifyArrays = stringifyArrays; + this.useBoundsAndSelectors = useBoundsAndSelectors; this.hook = hook == null ? NoOpPlannerHook.INSTANCE : hook; String sqlQueryId = (String) this.queryContext.get(QueryContexts.CTX_SQL_QUERY_ID); @@ -155,10 +163,12 @@ public static PlannerContext create( final DateTime utcNow; final DateTimeZone timeZone; final boolean stringifyArrays; + final boolean useBoundsAndSelectors; final Object stringifyParam = queryContext.get(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS); final Object tsParam = queryContext.get(CTX_SQL_CURRENT_TIMESTAMP); final Object tzParam = queryContext.get(CTX_SQL_TIME_ZONE); + final Object useBoundsAndSelectorsParam = queryContext.get(CTX_SQL_USE_BOUNDS_AND_SELECTORS); if (tsParam != null) { utcNow = new DateTime(tsParam, DateTimeZone.UTC); @@ -178,12 +188,19 @@ public static PlannerContext create( stringifyArrays = true; } + if (useBoundsAndSelectorsParam != null) { + useBoundsAndSelectors = Numbers.parseBoolean(useBoundsAndSelectorsParam); + } else { + useBoundsAndSelectors = DEFAULT_SQL_USE_BOUNDS_AND_SELECTORS; + } + return new PlannerContext( plannerToolbox, sql, plannerToolbox.plannerConfig().withOverrides(queryContext), utcNow.withZone(timeZone), stringifyArrays, + useBoundsAndSelectors, engine, queryContext, hook @@ -299,6 +316,11 @@ public boolean isStringifyArrays() return stringifyArrays; } + public boolean isUseBoundsAndSelectors() + { + return useBoundsAndSelectors; + } + public List getParameters() { return parameters; diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java index e831960a3907..62d235e5dcd0 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/run/NativeQueryMaker.java @@ -39,6 +39,7 @@ import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.segment.column.ColumnHolder; @@ -107,16 +108,20 @@ public QueryResponse runQuery(final DruidQuery druidQuery) OrDimFilter orDimFilter = (OrDimFilter) query.getFilter(); int numBoundFilters = 0; for (DimFilter filter : orDimFilter.getFields()) { - numBoundFilters += filter instanceof BoundDimFilter ? 1 : 0; - } - if (numBoundFilters > numFilters) { - String dimension = ((BoundDimFilter) (orDimFilter.getFields().get(0))).getDimension(); - throw new UOE(StringUtils.format( - "The number of values in the IN clause for [%s] in query exceeds configured maxNumericFilter limit of [%s] for INs. Cast [%s] values of IN clause to String", - dimension, - numFilters, - orDimFilter.getFields().size() - )); + if (filter instanceof BoundDimFilter) { + final BoundDimFilter bound = (BoundDimFilter) filter; + if (StringComparators.NUMERIC.equals(bound.getOrdering())) { + numBoundFilters++; + if (numBoundFilters > numFilters) { + throw new UOE(StringUtils.format( + "The number of values in the IN clause for [%s] in query exceeds configured maxNumericFilter limit of [%s] for INs. Cast [%s] values of IN clause to String", + bound.getDimension(), + numFilters, + orDimFilter.getFields().size() + )); + } + } + } } } } 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 2ba9ce6dc730..012e48ac9454 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 @@ -40,6 +40,7 @@ import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.math.expr.Evals; import org.apache.druid.query.DataSource; import org.apache.druid.query.Druids; import org.apache.druid.query.JoinDataSource; @@ -55,10 +56,13 @@ import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.RangeFilter; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.having.DimFilterHavingSpec; @@ -119,6 +123,7 @@ import java.io.IOException; import java.io.PrintStream; import java.util.Arrays; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -185,18 +190,27 @@ public static void setupNullValues() public static final String DUMMY_SQL_ID = "dummy"; public static final String PRETEND_CURRENT_TIME = "2000-01-01T00:00:00Z"; - private static final ImmutableMap.Builder DEFAULT_QUERY_CONTEXT_BUILDER = + + public static final Map QUERY_CONTEXT_DEFAULT = ImmutableMap.builder() .put(QueryContexts.CTX_SQL_QUERY_ID, DUMMY_SQL_ID) .put(PlannerContext.CTX_SQL_CURRENT_TIMESTAMP, "2000-01-01T00:00:00Z") .put(QueryContexts.DEFAULT_TIMEOUT_KEY, QueryContexts.DEFAULT_TIMEOUT_MILLIS) - .put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE); - public static final Map QUERY_CONTEXT_DEFAULT = DEFAULT_QUERY_CONTEXT_BUILDER.build(); + .put(QueryContexts.MAX_SCATTER_GATHER_BYTES_KEY, Long.MAX_VALUE) + .build(); public static final Map QUERY_CONTEXT_NO_STRINGIFY_ARRAY = - DEFAULT_QUERY_CONTEXT_BUILDER.put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false) - .put(PlannerContext.CTX_ENABLE_UNNEST, true) - .build(); + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(QueryContexts.CTX_SQL_STRINGIFY_ARRAYS, false) + .put(PlannerContext.CTX_ENABLE_UNNEST, true) + .build(); + + public static final Map QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY = + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .put(PlannerContext.CTX_SQL_USE_BOUNDS_AND_SELECTORS, false) + .build(); public static final Map QUERY_CONTEXT_DONT_SKIP_EMPTY_BUCKETS = ImmutableMap.of( QueryContexts.CTX_SQL_QUERY_ID, DUMMY_SQL_ID, @@ -240,8 +254,10 @@ public static void setupNullValues() ); public static final Map QUERY_CONTEXT_WITH_SUBQUERY_MEMORY_LIMIT = - DEFAULT_QUERY_CONTEXT_BUILDER.put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000") - .build(); + ImmutableMap.builder() + .putAll(QUERY_CONTEXT_DEFAULT) + .put(QueryContexts.MAX_SUBQUERY_BYTES_KEY, "100000") + .build(); // Add additional context to the given context map for when the // timeseries query has timestamp_floor expression on the timestamp dimension @@ -343,11 +359,66 @@ public static NotDimFilter not(DimFilter filter) return new NotDimFilter(filter); } - public static InDimFilter in(String dimension, List values, ExtractionFn extractionFn) + public static InDimFilter in(String dimension, Collection values, ExtractionFn extractionFn) { return new InDimFilter(dimension, values, extractionFn); } + public static DimFilter isNull(final String fieldName) + { + return isNull(fieldName, null); + } + + public static DimFilter isNull(final String fieldName, final ExtractionFn extractionFn) + { + if (NullHandling.sqlCompatible()) { + return new NullFilter(fieldName, extractionFn, null); + } + return selector(fieldName, NullHandling.defaultStringValue(), extractionFn); + } + + public static DimFilter notNull(final String fieldName) + { + return not(isNull(fieldName)); + } + + public static DimFilter equality(final String fieldName, final Object matchValue, final ColumnType matchValueType) + { + return equality(fieldName, matchValue, null, matchValueType); + } + + public static DimFilter equality( + final String fieldName, + final Object matchValue, + final ExtractionFn extractionFn, + final ColumnType matchValueType + ) + { + if (NullHandling.sqlCompatible()) { + return new EqualityFilter(fieldName, matchValueType, matchValue, extractionFn, null); + } + return selector(fieldName, Evals.asString(matchValue), extractionFn); + } + + /** + * Callers should use {@link #equality(String, Object, ColumnType)} or + * {@link #equality(String, Object, ExtractionFn, ColumnType)} instead of this method, since they will correctly use + * either a {@link EqualityFilter} or {@link SelectorDimFilter} depending on the value of + * {@link NullHandling#sqlCompatible()}, which determines the default of + * {@link PlannerContext#CTX_SQL_USE_BOUNDS_AND_SELECTORS} + */ + public static SelectorDimFilter selector(final String fieldName, final String value) + { + return selector(fieldName, value, null); + } + + /** + * Callers should use {@link #equality(String, Object, ColumnType)} or + * {@link #equality(String, Object, ExtractionFn, ColumnType)} instead of this method, since they will correctly use + * either a {@link EqualityFilter} or {@link SelectorDimFilter} depending on the value of + * {@link NullHandling#sqlCompatible()}, which determines the default of + * {@link PlannerContext#CTX_SQL_USE_BOUNDS_AND_SELECTORS} + */ public static SelectorDimFilter selector(final String fieldName, final String value, final ExtractionFn extractionFn) { return new SelectorDimFilter(fieldName, value, extractionFn); @@ -358,16 +429,41 @@ public static ExpressionDimFilter expressionFilter(final String expression) return new ExpressionDimFilter(expression, CalciteTests.createExprMacroTable()); } + /** + * This method should be used instead of {@link #equality(String, Object, ColumnType)} when the match value type + * does not match the column type. If {@link NullHandling#sqlCompatible()} is true, this method is equivalent to + * {@link #equality(String, Object, ColumnType)}. When false, this method uses + * {@link #numericSelector(String, String)} so that the equality comparison uses a bound filter to correctly match + * numerical types. + */ + public static DimFilter numericEquality( + final String fieldName, + final Object value, + final ColumnType matchValueType + ) + { + if (NullHandling.sqlCompatible()) { + return equality(fieldName, value, matchValueType); + } + return numericSelector(fieldName, String.valueOf(value)); + } + public static DimFilter numericSelector( final String fieldName, - final String value, - final ExtractionFn extractionFn + final String value ) { // We use Bound filters for numeric equality to achieve "10.0" = "10" - return bound(fieldName, value, value, false, false, extractionFn, StringComparators.NUMERIC); + return bound(fieldName, value, value, false, false, null, StringComparators.NUMERIC); } + /** + * Callers should use {@link #range(String, ColumnType, Object, Object, boolean, boolean)} or + * {@link #range(String, ColumnType, Object, Object, boolean, boolean, ExtractionFn)} instead of this method, since + * they will correctly use either a {@link RangeFilter} or {@link BoundDimFilter} depending on the value of + * {@link NullHandling#sqlCompatible()}, which determines the default of + * {@link PlannerContext#CTX_SQL_USE_BOUNDS_AND_SELECTORS} + */ public static BoundDimFilter bound( final String fieldName, final String lower, @@ -381,6 +477,11 @@ public static BoundDimFilter bound( return new BoundDimFilter(fieldName, lower, upper, lowerStrict, upperStrict, null, extractionFn, comparator); } + /** + * Callers should use {@link #timeRange(Object)} instead of this method, since it will correctly use either a + * {@link RangeFilter} or {@link BoundDimFilter} depending on the value of {@link NullHandling#sqlCompatible()}, + * which determines the default of {@link PlannerContext#CTX_SQL_USE_BOUNDS_AND_SELECTORS} + */ public static BoundDimFilter timeBound(final Object intervalObj) { final Interval interval = new Interval(intervalObj, ISOChronology.getInstanceUTC()); @@ -396,6 +497,59 @@ public static BoundDimFilter timeBound(final Object intervalObj) ); } + public static DimFilter range( + final String fieldName, + final ColumnType matchValueType, + final Object lower, + final Object upper, + final boolean lowerStrict, + final boolean upperStrict + ) + { + return range(fieldName, matchValueType, lower, upper, lowerStrict, upperStrict, null); + } + + public static DimFilter range( + final String fieldName, + final ColumnType matchValueType, + final Object lower, + final Object upper, + final boolean lowerStrict, + final boolean upperStrict, + final ExtractionFn extractionFn + ) + { + if (NullHandling.sqlCompatible()) { + return new RangeFilter(fieldName, matchValueType, lower, upper, lowerStrict, upperStrict, extractionFn, null); + } + return new BoundDimFilter( + fieldName, + Evals.asString(lower), + Evals.asString(upper), + lowerStrict, + upperStrict, + false, + extractionFn, + matchValueType.isNumeric() ? StringComparators.NUMERIC : StringComparators.LEXICOGRAPHIC + ); + } + + public static DimFilter timeRange(final Object intervalObj) + { + final Interval interval = new Interval(intervalObj, ISOChronology.getInstanceUTC()); + if (NullHandling.sqlCompatible()) { + return range( + ColumnHolder.TIME_COLUMN_NAME, + ColumnType.LONG, + interval.getStartMillis(), + interval.getEndMillis(), + false, + true + ); + } + return timeBound(intervalObj); + } + public static CascadeExtractionFn cascade(final ExtractionFn... fns) { return new CascadeExtractionFn(fns); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index da9eb754c433..1a43a240f170 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -41,12 +41,9 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.ExpressionDimFilter; 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.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.having.DimFilterHavingSpec; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -552,9 +549,9 @@ public void testArrayContainsFilter() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .filters( - new AndDimFilter( - new SelectorDimFilter("dim3", "a", null), - new SelectorDimFilter("dim3", "b", null) + and( + equality("dim3", "a", ColumnType.STRING), + equality("dim3", "b", ColumnType.STRING) ) ) .columns("dim3") @@ -578,7 +575,7 @@ public void testArrayContainsArrayOfOneElement() newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim3", "a", null)) + .filters(equality("dim3", "a", ColumnType.STRING)) .columns("dim3") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(5) @@ -1385,7 +1382,7 @@ public void testArrayToStringToMultiValueString() ColumnType.STRING_ARRAY ) ) - .setDimFilter(bound("v0", "0", null, true, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.LONG, 0L, null, true, false)) .setDimensions( dimensions( new DefaultDimensionSpec("v1", "_d0", ColumnType.STRING_ARRAY) @@ -1418,7 +1415,7 @@ public void testArrayAgg() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(not(selector("dim1", null, null))) + .filters(notNull("dim1")) .aggregators( aggregators( new ExpressionLambdaAggregatorFactory( @@ -1470,7 +1467,7 @@ public void testArrayAgg() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - selector("dim1", "shazbot", null) + equality("dim1", "shazbot", ColumnType.STRING) ) ) ) @@ -1894,7 +1891,7 @@ public void testArrayAggToString() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(not(selector("dim1", null, null))) + .filters(notNull("dim1")) .aggregators( aggregators( new ExpressionLambdaAggregatorFactory( @@ -2064,7 +2061,7 @@ public void testArrayAggAsArrayFromJoin() .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", null, null))) + .setDimFilter(notNull("dim1")) .setDimensions(new DefaultDimensionSpec("dim4", "_d0")) .setAggregatorSpecs( aggregators( @@ -2133,7 +2130,7 @@ public void testArrayAggGroupByArrayAggFromSubquery() StringComparators.LEXICOGRAPHIC ), 5, querySegmentSpec(Filtration.eternity()), - new NotDimFilter(new SelectorDimFilter("dim1", null, null)), + notNull("dim1"), Granularities.ALL, aggregators(new ExpressionLambdaAggregatorFactory( "a0", @@ -2420,7 +2417,7 @@ public void testArrayAggArrayContainsSubquery() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(not(selector("dim1", null, null))) + .filters(notNull("dim1")) .aggregators( aggregators( new ExpressionLambdaAggregatorFactory( @@ -2500,7 +2497,7 @@ public void testArrayAggGroupByArrayContainsSubquery() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(not(selector("dim1", null, null))) + .filters(notNull("dim1")) .aggregators( aggregators( new ExpressionLambdaAggregatorFactory( @@ -2998,7 +2995,7 @@ public void testUnnestWithGroupByHaving() .setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setAggregatorSpecs(new CountAggregatorFactory("a0")) - .setHavingSpec(new DimFilterHavingSpec(selector("a0", "1", null), true)) + .setHavingSpec(new DimFilterHavingSpec(equality("a0", 1L, ColumnType.LONG), true)) .setContext(QUERY_CONTEXT_UNNEST) .build() ), @@ -3125,7 +3122,7 @@ public void testUnnestWithFilters() .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) - .filters(new SelectorDimFilter("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .columns("dim3") .context(QUERY_CONTEXT_UNNEST) .build() @@ -3177,8 +3174,8 @@ public void testUnnestWithFiltersInsideAndOutside() .legacy(false) .filters( and( - selector("dim2", "a", null), - not(selector("dim1", "foo", null)) + equality("dim2", "a", ColumnType.STRING), + not(equality("dim1", "foo", ColumnType.STRING)) ) ) .columns("dim3") @@ -3186,7 +3183,7 @@ public void testUnnestWithFiltersInsideAndOutside() .build() ), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - not(selector("j0.unnest", "b", null)) + not(equality("j0.unnest", "b", ColumnType.STRING)) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -3232,8 +3229,8 @@ public void testUnnestWithFiltersOutside() .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .filters( and( - selector("dim2", "a", null), - not(selector("dim1", "foo", null)) + equality("dim2", "a", ColumnType.STRING), + not(equality("dim1", "foo", ColumnType.STRING)) ) ) .legacy(false) @@ -3569,7 +3566,7 @@ public void testUnnestWithINFiltersWithLeftRewrite() )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) - .filters(bound("m1", null, "10", false, true, null, StringComparators.NUMERIC)) + .filters(range("m1", ColumnType.LONG, null, 10L, false, true)) .legacy(false) .context(QUERY_CONTEXT_UNNEST) .columns(ImmutableList.of("j0.unnest")) @@ -3655,7 +3652,7 @@ public void testUnnestWithNotFiltersOnUnnestedColumn() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - not(selector("j0.unnest", "d", null)) + not(equality("j0.unnest", "d", ColumnType.STRING)) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -3699,7 +3696,7 @@ public void testUnnestWithSelectorFiltersOnSelectedColumn() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - selector("j0.unnest", "b", null) + equality("j0.unnest", "b", ColumnType.STRING) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -3728,7 +3725,9 @@ public void testUnnestWithSelectorFiltersOnVirtualColumn() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "array(\"m1\",\"m2\")", ColumnType.FLOAT_ARRAY), - selector("j0.unnest", "1", null) + NullHandling.replaceWithDefault() + ? selector("j0.unnest", "1") + : equality("j0.unnest", 1.0, ColumnType.FLOAT) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -3789,7 +3788,7 @@ public void testUnnestWithMultipleAndFiltersOnSelectedColumns() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - selector("j0.unnest", "b", null) + equality("j0.unnest", "b", ColumnType.STRING) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -3797,8 +3796,8 @@ public void testUnnestWithMultipleAndFiltersOnSelectedColumns() .context(QUERY_CONTEXT_UNNEST) .filters( and( - bound("m1", null, "10", false, true, null, StringComparators.NUMERIC), - bound("m2", null, "10", false, true, null, StringComparators.NUMERIC) + range("m1", ColumnType.LONG, null, 10L, false, true), + range("m2", ColumnType.LONG, null, 10L, false, true) ) ) .columns(ImmutableList.of("j0.unnest")) @@ -3832,8 +3831,8 @@ public void testUnnestWithMultipleOrFiltersOnSelectedColumns() .context(QUERY_CONTEXT_UNNEST) .filters( or( - selector("j0.unnest", "b", null), - bound("m1", null, "2", false, true, null, StringComparators.NUMERIC) + equality("j0.unnest", "b", ColumnType.STRING), + range("m1", ColumnType.LONG, null, 2L, false, true) ) ) .columns(ImmutableList.of("j0.unnest")) @@ -3862,7 +3861,7 @@ public void testUnnestWithMultipleAndFiltersOnSelectedUnnestedColumns() expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), and( new InDimFilter("j0.unnest", ImmutableSet.of("a", "b"), null), - bound("j0.unnest", null, "e", false, true, null, StringComparators.LEXICOGRAPHIC) + range("j0.unnest", ColumnType.STRING, null, "e", false, true) ) )) .intervals(querySegmentSpec(Filtration.eternity())) @@ -3924,8 +3923,8 @@ public void testUnnestWithMultipleOrFiltersOnVariationsOfUnnestedColumns() new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), or( - new ExpressionDimFilter("(strlen(\"j0.unnest\") < 2)", TestExprMacroTable.INSTANCE), - selector("j0.unnest", "d", null) + expressionFilter("(strlen(\"j0.unnest\") < 2)"), + equality("j0.unnest", "d", ColumnType.STRING) ) )) .intervals(querySegmentSpec(Filtration.eternity())) @@ -3978,8 +3977,8 @@ public void testUnnestWithMultipleOrFiltersOnSelectedNonUnnestedColumns() .context(QUERY_CONTEXT_UNNEST) .filters( or( - bound("m1", null, "2", false, true, null, StringComparators.NUMERIC), - bound("m2", null, "2", false, true, null, StringComparators.NUMERIC) + range("m1", ColumnType.LONG, null, 2L, false, true), + range("m2", ColumnType.LONG, null, 2L, false, true) ) ) .columns(ImmutableList.of("j0.unnest")) @@ -4013,7 +4012,7 @@ public void testUnnestWithMultipleOrFiltersOnSelectedVirtualColumns() .context(QUERY_CONTEXT_UNNEST) .filters( or( - bound("m1", null, "2", false, true, null, StringComparators.NUMERIC), + range("m1", ColumnType.LONG, null, 2L, false, true), new InDimFilter("j0.unnest", ImmutableSet.of("a", "aa"), null) ) ) @@ -4049,8 +4048,8 @@ public void testUnnestWithMultipleOrFiltersOnUnnestedColumnsAndOnOriginalColumn( .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .filters( or( - selector("j0.unnest", "b", null), - selector("dim3", "d", null) + equality("j0.unnest", "b", ColumnType.STRING), + equality("dim3", "d", ColumnType.STRING) ) ) .legacy(false) @@ -4085,8 +4084,8 @@ public void testUnnestWithMultipleOrFiltersOnUnnestedColumnsAndOnOriginalColumnD .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .filters( or( - selector("dim3", "b", null), - selector("j0.unnest", "a", null) + equality("dim3", "b", ColumnType.STRING), + equality("j0.unnest", "a", ColumnType.STRING) ) ) .legacy(false) @@ -4148,7 +4147,7 @@ public void testUnnestWithGroupByHavingSelector() .setContext(QUERY_CONTEXT_UNNEST) .setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING)) .setGranularity(Granularities.ALL) - .setDimFilter(selector("j0.unnest", "b", null)) + .setDimFilter(equality("j0.unnest", "b", ColumnType.STRING)) .setAggregatorSpecs(new CountAggregatorFactory("a0")) .setContext(QUERY_CONTEXT_UNNEST) .build() @@ -4236,7 +4235,7 @@ public void testUnnestWithGroupByHavingWithWhereOnAggCol() .setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setAggregatorSpecs(new CountAggregatorFactory("a0")) - .setHavingSpec(new DimFilterHavingSpec(selector("a0", "1", null), true)) + .setHavingSpec(new DimFilterHavingSpec(equality("a0", 1L, ColumnType.LONG), true)) .setContext(QUERY_CONTEXT_UNNEST) .build() ), @@ -4267,7 +4266,7 @@ public void testUnnestWithGroupByHavingWithWhereOnUnnestCol() .setDimensions(new DefaultDimensionSpec("j0.unnest", "_d0", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setAggregatorSpecs(new CountAggregatorFactory("a0")) - .setDimFilter(selector("j0.unnest", "a", null)) + .setDimFilter(equality("j0.unnest", "a", ColumnType.STRING)) .setContext(QUERY_CONTEXT_UNNEST) .build() ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java index 577f46a2e993..89b09872d402 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteCorrelatedQueryTest.java @@ -87,7 +87,7 @@ public void testCorrelatedSubquery(Map queryContext) ColumnType.LONG, TestExprMacroTable.INSTANCE )) - .setDimFilter(not(selector("country", null, null))) + .setDimFilter(notNull("country")) .setDimensions( new DefaultDimensionSpec( "v0", @@ -132,7 +132,7 @@ public void testCorrelatedSubquery(Map queryContext) ? new CountAggregatorFactory("_a0:count") : new FilteredAggregatorFactory( new CountAggregatorFactory("_a0:count"), - not(selector("a0", null, null)) + notNull("a0") ) ) .setPostAggregatorSpecs(Collections.singletonList(new ArithmeticPostAggregator( @@ -205,7 +205,7 @@ public void testCorrelatedSubqueryWithLeftFilter(Map queryContex ColumnType.LONG, TestExprMacroTable.INSTANCE )) - .setDimFilter(not(selector("country", null, null))) + .setDimFilter(notNull("country")) .setDimensions( new DefaultDimensionSpec( "v0", @@ -243,7 +243,7 @@ public void testCorrelatedSubqueryWithLeftFilter(Map queryContex makeColumnExpression("j0._d0") ), JoinType.LEFT, - selector("city", "B", null) + equality("city", "B", ColumnType.STRING) ) ) .setQuerySegmentSpec(querySegmentSpec(Intervals.of( @@ -283,7 +283,11 @@ public void testCorrelatedSubqueryWithLeftFilter_leftDirectAccessDisabled(Map ) .setAggregatorSpecs(new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("user", null, null)) + notNull("user") )) .setDimFilter(and( - selector("city", "A", null), - not(selector("country", null, null)) + equality("city", "A", ColumnType.STRING), + notNull("country") )) .setContext( withTimestampResultContext( @@ -429,7 +433,7 @@ public void testCorrelatedSubqueryWithCorrelatedQueryFilter(Map makeColumnExpression("j0._d0") ), JoinType.LEFT, - selector("city", "B", null) + equality("city", "B", ColumnType.STRING) ) ) .setQuerySegmentSpec(querySegmentSpec(Intervals.ETERNITY)) @@ -491,11 +495,11 @@ public void testCorrelatedSubqueryWithCorrelatedQueryFilter_Scan(Map}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"; - final String explanation = "[" + final String legacyExplanation = NullHandling.replaceWithDefault() + ? + "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" + + " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" + : + "DruidOuterQueryRel(query=[{\"queryType\":\"timeseries\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n" + + " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"null\",\"column\":\"dim1\"}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"; + final String explanation = NullHandling.replaceWithDefault() ? + "[" + "{\"query\":{\"queryType\":\"groupBy\"," + "\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"table\",\"name\":\"foo\"},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}},\"rightPrefix\":\"j0.\",\"condition\":\"(substring(\\\"dim2\\\", 0, 1) == \\\"j0.d0\\\")\",\"joinType\":\"INNER\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}}," + "\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]}," @@ -127,7 +146,8 @@ public void testExplainExactCountDistinctOfSemiJoinResult() + "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}," + "\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}]," + "\"columnMappings\":[{\"queryColumn\":\"a0\",\"outputColumn\":\"EXPR$0\"}]" - + "}]"; + + "}]" + : "[{\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"join\",\"left\":{\"type\":\"table\",\"name\":\"foo\"},\"right\":{\"type\":\"query\",\"query\":{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"null\",\"column\":\"dim1\"}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}},\"rightPrefix\":\"j0.\",\"condition\":\"(substring(\\\"dim2\\\", 0, 1) == \\\"j0.d0\\\")\",\"joinType\":\"INNER\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}},\"signature\":[{\"name\":\"a0\",\"type\":\"LONG\"}],\"columnMappings\":[{\"queryColumn\":\"a0\",\"outputColumn\":\"EXPR$0\"}]}]"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; @@ -230,27 +250,35 @@ public void testExplainMultipleTopLevelUnionAllQueries() final String query = "EXPLAIN PLAN FOR SELECT dim1 FROM druid.foo\n" + "UNION ALL (SELECT dim1 FROM druid.foo WHERE dim1 = '42'\n" + "UNION ALL SELECT dim1 FROM druid.foo WHERE dim1 = '44')"; - final String legacyExplanation = "DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidUnionRel(limit=[-1])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" - + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"; - final String explanation = "[" - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}," - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}," - + "{" - + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," - + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," - + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" - + "}]"; + final String legacyExplanation = NullHandling.replaceWithDefault() + ? "DruidUnionRel(limit=[-1])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" + + " DruidUnionRel(limit=[-1])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" + : "DruidUnionRel(limit=[-1])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" + + " DruidUnionRel(limit=[-1])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n" + + " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{dim1:STRING}])\n"; + final String explanation = NullHandling.replaceWithDefault() + ? "[" + + "{" + + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," + + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," + + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" + + "}," + + "{" + + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," + + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," + + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" + + "}," + + "{" + + "\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}," + + "\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}]," + + "\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]" + + "}]" + : "[{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"42\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]},{\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"filter\":{\"type\":\"equals\",\"column\":\"dim1\",\"matchValueType\":\"STRING\",\"matchValue\":\"44\"},\"columns\":[\"dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}},\"signature\":[{\"name\":\"dim1\",\"type\":\"STRING\"}],\"columnMappings\":[{\"queryColumn\":\"dim1\",\"outputColumn\":\"dim1\"}]}]"; final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]"; final String attributes = "{\"statementType\":\"SELECT\"}"; testQuery( diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java index 1a19f9ddf70e..a92f53cb727e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteInsertDmlTest.java @@ -99,7 +99,7 @@ public void testInsertFromViewA() .dataSource("foo") .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns(expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ColumnType.STRING)) - .filters(selector("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .columns("v0") .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT) .build() @@ -129,7 +129,7 @@ public void testInsertFromViewC() newScanQueryBuilder() .dataSource("foo") .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .columns("dim1", "dim2") .context(PARTITIONED_BY_ALL_TIME_QUERY_CONTEXT) .build() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index c4ff4a17a3f4..3a08e07a43ca 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -63,13 +63,7 @@ import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.extraction.SubstringDimExtractionFn; -import org.apache.druid.query.filter.AndDimFilter; -import org.apache.druid.query.filter.BoundDimFilter; -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.OrDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -102,10 +96,8 @@ import org.junit.runner.RunWith; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; @@ -216,7 +208,7 @@ public void testExactTopNOnInnerJoinWithLimit() GroupByQuery.builder() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(new NotDimFilter(new SelectorDimFilter("dim4", "a", null))) + .setDimFilter(not(equality("dim4", "a", ColumnType.STRING))) .setDataSource(new TableDataSource("numfoo")) .setDimensions(new DefaultDimensionSpec("dim4", "_d0")) .setContext(context) @@ -290,7 +282,7 @@ public void testJoinOuterGroupByAndSubqueryHasLimit() new DoubleSumAggregatorFactory("a0:sum", "m2"), new FilteredAggregatorFactory( new CountAggregatorFactory("a0:count"), - not(selector("m2", null, null)) + notNull("m2") ) ) ) @@ -373,7 +365,7 @@ public void testJoinOuterGroupByAndSubqueryNoLimit(Map queryCont new DoubleSumAggregatorFactory("a0:sum", "m2"), new FilteredAggregatorFactory( new CountAggregatorFactory("a0:count"), - not(selector("m2", null, null)) + notNull("m2") ) ) ) @@ -460,7 +452,7 @@ public void testJoinWithLimitBeforeJoining() new DoubleSumAggregatorFactory("a0:sum", "m2"), new FilteredAggregatorFactory( new CountAggregatorFactory("a0:count"), - not(selector("m2", null, null)) + notNull("m2") ) ) ) @@ -517,7 +509,7 @@ public void testJoinOnTimeseriesWithFloorOnTime() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Intervals.of("1994-04-29/2020-01-11T00:00:00.001Z"))) - .filters(selector("dim3", "b", null)) + .filters(equality("dim3", "b", ColumnType.STRING)) .granularity(new PeriodGranularity(Period.hours(1), null, DateTimeZone.UTC)) .aggregators(aggregators( new FloatMinAggregatorFactory("a0", "m1") @@ -579,7 +571,7 @@ public void testJoinOnGroupByInsteadOfTimeseriesWithFloorOnTime() ColumnType.LONG ) ) - .setDimFilter(selector("dim3", "b", null)) + .setDimFilter(equality("dim3", "b", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec( "v0", @@ -642,7 +634,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorWithValueFilterPushdownMa ) ) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("j0.v", "123", null)) + .setDimFilter(equality("j0.v", "123", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("j0.k", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) @@ -679,7 +671,12 @@ public void testFilterAndGroupByLookupUsingJoinOperatorAllowNulls(Map queryContext) ) ) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(not(selector("j0.v", "xa", null))) + .setDimFilter(not(equality("j0.v", "xa", ColumnType.STRING))) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("j0.v", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) @@ -856,7 +853,7 @@ public void testFilterAndGroupByLookupUsingJoinOperator(Map quer ) ) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("j0.v", "xa", null)) + .setDimFilter(equality("j0.v", "xa", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("j0.k", "d0"))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) @@ -903,7 +900,12 @@ public void testFilterAndGroupByLookupUsingPostAggregationJoinOperator(Map queryConte ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("j0.v", "xa", null)) + .filters(equality("j0.v", "xa", ColumnType.STRING)) .columns("dim1") .context(queryContext) .build() @@ -2365,7 +2367,12 @@ public void testSelectOnLookupUsingLeftJoinOperator(Map queryCon ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(or(not(selector("j0.v", "xxx", null)), selector("j0.v", null, null))) + .filters( + or( + not(equality("j0.v", "xxx", ColumnType.STRING)), + isNull("j0.v") + ) + ) .columns("dim1", "j0.k", "j0.v") .context(queryContext) .build() @@ -2408,7 +2415,12 @@ public void testSelectOnLookupUsingRightJoinOperator(Map queryCo ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(or(not(selector("j0.v", "xxx", null)), selector("j0.v", null, null))) + .filters( + or( + not(equality("j0.v", "xxx", ColumnType.STRING)), + isNull("j0.v") + ) + ) .columns("dim1", "j0.k", "j0.v") .context(queryContext) .build() @@ -2446,7 +2458,12 @@ public void testSelectOnLookupUsingFullJoinOperator(Map queryCon ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(or(not(selector("j0.v", "xxx", null)), selector("j0.v", null, null))) + .filters( + or( + not(equality("j0.v", "xxx", ColumnType.STRING)), + isNull("j0.v") + ) + ) .columns("cnt", "dim1", "j0.k", "j0.v", "m1") .context(queryContext) .build() @@ -2553,7 +2570,7 @@ public void testNotInAggregationSubquery(Map queryContext) NullHandling.sqlCompatible() ? new FilteredAggregatorFactory( new CountAggregatorFactory("_a1"), - not(selector("a0", null, null)) + notNull("a0") ) : new CountAggregatorFactory("_a1") ) @@ -2583,8 +2600,8 @@ public void testNotInAggregationSubquery(Map queryContext) .setGranularity(Granularities.ALL) .setDimFilter( or( - selector("j0._a0", "0", null), - and(selector("_j0.p0", null, null), expressionFilter("(\"j0._a1\" >= \"j0._a0\")")) + equality("j0._a0", 0L, ColumnType.LONG), + and(isNull("_j0.p0"), expressionFilter("(\"j0._a1\" >= \"j0._a0\")")) ) ) .setDimensions(dimensions(new DefaultDimensionSpec("__time", "d0", ColumnType.LONG))) @@ -2623,7 +2640,9 @@ public void testUsingSubqueryWithExtractionFns(Map queryContext) .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) + .setDimFilter( + not(equality("dim1", "", ColumnType.STRING)) + ) .setDimensions( dimensions(new ExtractionDimensionSpec( "dim1", @@ -2681,7 +2700,7 @@ public void testInnerJoinWithIsNullFilter(Map queryContext) ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim2", null, null)) + .filters(isNull("dim2")) .columns("dim1", "j0.v") .build() ), @@ -2758,7 +2777,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map ) ) ) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -2776,7 +2795,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map ) ) ) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -2791,7 +2810,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter(Map .virtualColumns(expressionVirtualColumn("_v0", "'10.1'", ColumnType.STRING)) .intervals(querySegmentSpec(Filtration.eternity())) .columns("__time", "_v0") - .filters(new SelectorDimFilter("v0", "10.1", null)) + .filters(equality("v0", "10.1", ColumnType.STRING)) .context(queryContext) .build() ), @@ -2829,7 +2848,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAcces ) ) ) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -2839,7 +2858,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithTimeFilter_withLeftDirectAcces "j0.", equalsCondition(makeExpression("'10.1'"), makeColumnExpression("j0.v0")), JoinType.LEFT, - selector("dim1", "10.1", null) + equality("dim1", "10.1", ColumnType.STRING) ) ) .intervals(querySegmentSpec( @@ -2878,7 +2897,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere(Map newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -2889,7 +2908,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere(Map newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .columns(ImmutableList.of("dim1")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -2902,7 +2921,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere(Map ) .virtualColumns(expressionVirtualColumn("_v0", "'10.1'", ColumnType.STRING)) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("v0", "10.1", null)) + .filters(equality("v0", "10.1", ColumnType.STRING)) .columns("__time", "_v0") .context(queryContext) .build() @@ -2934,7 +2953,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAcces newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .columns(ImmutableList.of("dim1")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -2946,7 +2965,7 @@ public void testLeftJoinOnTwoInlineDataSourcesWithOuterWhere_withLeftDirectAcces makeColumnExpression("j0.dim1") ), JoinType.LEFT, - selector("dim1", "10.1", null) + equality("dim1", "10.1", ColumnType.STRING) ) ) .intervals(querySegmentSpec(Filtration.eternity())) @@ -2980,7 +2999,7 @@ public void testLeftJoinOnTwoInlineDataSources(Map queryContext) newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -2991,7 +3010,7 @@ public void testLeftJoinOnTwoInlineDataSources(Map queryContext) newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .columns(ImmutableList.of("dim1")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -3035,7 +3054,7 @@ public void testLeftJoinOnTwoInlineDataSources_withLeftDirectAccess(Map queryContext newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .virtualColumns(expressionVirtualColumn("v0", "\'10.1\'", ColumnType.STRING)) .columns(ImmutableList.of("__time", "v0")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -3194,7 +3213,7 @@ public void testInnerJoinOnTwoInlineDataSources(Map queryContext newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new SelectorDimFilter("dim1", "10.1", null)) + .filters(equality("dim1", "10.1", ColumnType.STRING)) .columns(ImmutableList.of("dim1")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(queryContext) @@ -3241,7 +3260,7 @@ public void testGroupByOverGroupByOverInnerJoinOnTwoInlineDataSources(Map queryConte ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1", "j0.d0") - .filters(new NotDimFilter(new SelectorDimFilter("j0.d0", null, null))) + .filters(notNull("j0.d0")) .context(queryContext) .build(); @@ -3580,7 +3597,7 @@ public void testLeftJoinSubqueryWithSelectorFilter(Map queryCont ) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1", "j0.d0") - .filters(selector("j0.d0", "abc", null)) + .filters(equality("j0.d0", "abc", ColumnType.STRING)) .context(queryContext) .build() ), @@ -3718,7 +3735,7 @@ public void testSemiJoinWithOuterTimeExtractScan() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) - .setDimFilter(selector("dim1", "def", null)) + .setDimFilter(equality("dim1", "def", ColumnType.STRING)) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -3731,7 +3748,9 @@ public void testSemiJoinWithOuterTimeExtractScan() .virtualColumns( expressionVirtualColumn("v0", "timestamp_extract(\"__time\",'MONTH','UTC')", ColumnType.LONG) ) - .filters(not(selector("dim1", "", null))) + .filters( + not(equality("dim1", "", ColumnType.STRING)) + ) .columns("dim1", "v0") .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3773,7 +3792,7 @@ public void testTwoSemiJoinsSimultaneously(Map queryContext) .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .bound(TimeBoundaryQuery.MAX_TIME) - .filters(selector("cnt", "1", null)) + .filters(equality("cnt", 1L, ColumnType.LONG)) .context(maxTimeQueryContext) .build() ), @@ -3786,7 +3805,7 @@ public void testTwoSemiJoinsSimultaneously(Map queryContext) .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .bound(TimeBoundaryQuery.MAX_TIME) - .filters(not(selector("cnt", "2", null))) + .filters(not(equality("cnt", 2L, ColumnType.LONG))) .context(maxTimeQueryContext) .build() ), @@ -3864,7 +3883,7 @@ public void testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery(Map= \"_j0._a0\")") ) ) @@ -3969,7 +3988,7 @@ public void testSemiAndAntiJoinSimultaneouslyUsingExplicitJoins(Map queryContext) .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) + .setDimFilter( + not( + NullHandling.replaceWithDefault() + ? isNull("dim1") + : equality("dim1", "", ColumnType.STRING) + ) + ) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4466,7 +4497,7 @@ public void testUsingSubqueryAsPartOfAndFilter(Map queryContext) ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "xxx", null))) + .setDimFilter(not(equality("dim1", "xxx", ColumnType.STRING))) .setDimensions( dimensions( new DefaultDimensionSpec("dim1", "d0"), @@ -4549,11 +4580,11 @@ public void testUsingSubqueryAsPartOfOrFilter(Map queryContext) .setGranularity(Granularities.ALL) .setDimFilter( or( - selector("dim1", "xxx", null), + equality("dim1", "xxx", ColumnType.STRING), and( - not(selector("j0.a0", "0", null)), - not(selector("_j0.d1", null, null)), - not(selector("dim2", null, null)) + not(equality("j0.a0", 0L, ColumnType.LONG)), + notNull("_j0.d1"), + notNull("dim2") ) ) ) @@ -4638,7 +4669,7 @@ public void testNestedGroupByOnInlineDataSourceWithFilter(Map qu ) .setGranularity(Granularities.ALL) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim1", "def", null)) + .setDimFilter(equality("dim1", "def", ColumnType.STRING)) .setDimensions( dimensions( new DefaultDimensionSpec("v0", "d0") @@ -4747,7 +4778,7 @@ public void testCountOnSemiJoinSingleColumn(Map queryContext) .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter( - selector("dim1", "10.1", null) + equality("dim1", "10.1", ColumnType.STRING) ) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) @@ -5014,82 +5045,82 @@ public void testInnerJoinWithFilterPushdownAndManyFiltersEmptyResults(Map q .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setVirtualColumns(expressionVirtualColumn("v0", "1", ColumnType.LONG)) - .setDimFilter(selector("m2", "A", null)) + .setDimFilter(equality("m2", "A", ColumnType.STRING)) .setDimensions( new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) ) @@ -5368,14 +5399,14 @@ public void testRegressionFilteredAggregatorsSubqueryJoins(Map q new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), and( - not(selector("_j0.d1", null, null)), - not(selector("dim1", null, null)) + notNull("_j0.d1"), + notNull("dim1") ), "a0" ), new FilteredAggregatorFactory( new FloatMinAggregatorFactory("a1", "m1"), - selector("__j0.d0", null, null), + isNull("__j0.d0"), "a1" ) ) @@ -5424,10 +5455,12 @@ public void testRegressionFilteredAggregatorsSubqueryJoins(Map q new QueryDataSource( new TopNQueryBuilder().dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new InDimFilter( - "m2", - new HashSet<>(Arrays.asList(null, "A")) - )) + .filters( + or( + equality("m2", "A", ColumnType.STRING), + isNull("m2") + ) + ) .virtualColumns(expressionVirtualColumn( "v0", "notnull(\"m2\")", @@ -5452,19 +5485,19 @@ public void testRegressionFilteredAggregatorsSubqueryJoins(Map q new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), and( - not(selector("_j0.d1", null, null)), - not(selector("dim1", null, null)) + notNull("_j0.d1"), + notNull("dim1") ), "a0" ), new FilteredAggregatorFactory( new FloatMinAggregatorFactory("a1", "m1"), or( - selector("__j0.a0", null, null), + isNull("__j0.a0"), not( or( not(expressionFilter("\"__j0.d0\"")), - not(selector("__j0.d0", null, null)) + notNull("__j0.d0") ) ) ), @@ -5581,7 +5614,14 @@ public void testJoinsWithTwoConditions() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDataSource(new TableDataSource(CalciteTests.DATASOURCE1)) - .setDimFilter(in("m1", ImmutableList.of("1", "2"), null)) + .setDimFilter( + NullHandling.replaceWithDefault() + ? in("m1", ImmutableList.of("1", "2"), null) + : or( + equality("m1", 1.0, ColumnType.FLOAT), + equality("m1", 2.0, ColumnType.FLOAT) + ) + ) .setDimensions(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT)) .setAggregatorSpecs(aggregators(new LongMaxAggregatorFactory("a0", "__time"))) .setContext(context) @@ -5625,10 +5665,21 @@ public void testJoinsWithThreeConditions() .setGranularity(Granularities.ALL) .setDataSource(new TableDataSource(CalciteTests.DATASOURCE1)) .setDimFilter( - and( + NullHandling.replaceWithDefault() + ? and( in("m1", ImmutableList.of("1", "2"), null), in("m2", ImmutableList.of("1", "2"), null) + ) + : and( + or( + equality("m1", 1.0, ColumnType.FLOAT), + equality("m1", 2.0, ColumnType.FLOAT) + ), + or( + equality("m2", 1.0, ColumnType.DOUBLE), + equality("m2", 2.0, ColumnType.DOUBLE) ) + ) ) .setDimensions( new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT), @@ -5693,7 +5744,7 @@ public void testJoinWithInputRefCondition() .aggregators(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - new SelectorDimFilter("j0.d1", null, null) + isNull("j0.d1") ) )) .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0")) @@ -5714,7 +5765,7 @@ public void testJoinWithInputRefCondition() new CountAggregatorFactory("a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("a1"), - not(selector("m1", null, null)), + notNull("m1"), "a1" ) ) @@ -5752,9 +5803,9 @@ public void testJoinWithInputRefCondition() new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), or( - new SelectorDimFilter("j0.a0", "0", null), + equality("j0.a0", 0L, ColumnType.LONG), and( - selector("_j0.d1", null, null), + isNull("_j0.d1"), expressionFilter("(\"j0.a1\" >= \"j0.a0\")") ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java index f17ef78c1752..d4f25101545c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteMultiValueStringQueryTest.java @@ -33,12 +33,10 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; -import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.LikeDimFilter; import org.apache.druid.query.filter.OrDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -161,7 +159,7 @@ public void testMultiValueStringWorksLikeStringGroupByWithFilter() new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING) ) ) - .setDimFilter(selector("v0", "bfoo", null)) + .setDimFilter(equality("v0", "bfoo", ColumnType.STRING)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec(new DefaultLimitSpec( ImmutableList.of(new OrderByColumnSpec( @@ -248,7 +246,7 @@ public void testMultiValueStringWorksLikeStringScanWithFilter() .dataSource(CalciteTests.DATASOURCE3) .eternityInterval() .virtualColumns(expressionVirtualColumn("v0", "concat(\"dim3\",'foo')", ColumnType.STRING)) - .filters(selector("v0", "bfoo", null)) + .filters(equality("v0", "bfoo", ColumnType.STRING)) .columns(ImmutableList.of("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -316,9 +314,9 @@ public void testMultiValueStringContainsFilter() .dataSource(CalciteTests.DATASOURCE3) .eternityInterval() .filters( - new AndDimFilter( - new SelectorDimFilter("dim3", "a", null), - new SelectorDimFilter("dim3", "b", null) + and( + equality("dim3", "a", ColumnType.STRING), + equality("dim3", "b", ColumnType.STRING) ) ) .columns("dim3") @@ -342,7 +340,7 @@ public void testMultiValueStringContainsArrayOfOneElement() newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .eternityInterval() - .filters(new SelectorDimFilter("dim3", "a", null)) + .filters(equality("dim3", "a", ColumnType.STRING)) .columns("dim3") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(5) @@ -1028,7 +1026,7 @@ public void testMultiValueStringToStringToMultiValueString() ColumnType.STRING ) ) - .setDimFilter(bound("v0", "0", null, true, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.LONG, 0L, null, true, false)) .setDimensions(dimensions(new DefaultDimensionSpec("v1", "_d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setLimitSpec(new DefaultLimitSpec( @@ -1209,7 +1207,7 @@ public void testStringToMVOfStringAgg() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("dim1", null, null)) + notNull("dim1") ), new CountAggregatorFactory("a1") ) @@ -1691,7 +1689,7 @@ public void testFilterOnMultiValueListFilterNoMatch() true ) ) - .setDimFilter(selector("v0", "a", null)) + .setDimFilter(equality("v0", "a", ColumnType.STRING)) .setDimensions( dimensions( new DefaultDimensionSpec("dim3", "_d0", ColumnType.STRING) @@ -1734,7 +1732,7 @@ public void testFilterOnMultiValueListFilterMatch() true ) ) - .setDimFilter(selector("v0", "b", null)) + .setDimFilter(equality("v0", "b", ColumnType.STRING)) .setDimensions( dimensions( new DefaultDimensionSpec("dim3", "_d0", ColumnType.STRING) @@ -2105,9 +2103,9 @@ public void testMultiValueStringOverlapFilterCoalesceSingleValue() ) ) .filters( - new OrDimFilter( - new InDimFilter("dim3", ImmutableSet.of("a", "b", "other")), - new SelectorDimFilter("dim3", null, null) + or( + in("dim3", ImmutableSet.of("a", "b", "other"), null), + isNull("dim3") ) ) .columns("v0") @@ -2152,11 +2150,11 @@ public void testMultiValueStringOverlapFilterCoalesceSingleValueOtherColumn() ) ) .filters( - new OrDimFilter( - new InDimFilter("dim3", ImmutableSet.of("a", "b", "other")), - new AndDimFilter( - new InDimFilter("dim2", ImmutableSet.of("a", "b", "other")), - new SelectorDimFilter("dim3", null, null) + or( + in("dim3", ImmutableSet.of("a", "b", "other"), null), + and( + in("dim2", ImmutableSet.of("a", "b", "other"), null), + isNull("dim3") ) ) ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index 628d7dc3246f..db37a763ab7f 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -49,10 +49,10 @@ import org.apache.druid.query.aggregation.FilteredAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.LikeDimFilter; -import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.scan.ScanQuery; @@ -929,17 +929,52 @@ public void testJsonValueArrays() ImmutableList.of( new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), null}, new Object[]{null, null, null, null}, - new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2D, 3.3D, 4.0D), Arrays.asList(1L, 2L)}, + new Object[]{ + Arrays.asList("d", "e"), + Arrays.asList(1L, 4L), + Arrays.asList(2.2D, 3.3D, 4.0D), + Arrays.asList(1L, 2L) + }, new Object[]{Arrays.asList("a", "b"), null, null, Collections.singletonList(1L)}, - new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), Arrays.asList(1L, 2L, null)}, - new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1D, 3.3D), Collections.singletonList(1L)}, - new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3D, 4.4D, 5.5D), null}, + new Object[]{ + Arrays.asList("a", "b"), + Arrays.asList(1L, 2L, 3L), + Arrays.asList(1.1D, 2.2D, 3.3D), + Arrays.asList(1L, 2L, null) + }, + new Object[]{ + Arrays.asList("b", "c"), + Arrays.asList(1L, 2L, 3L, 4L), + Arrays.asList(1.1D, 3.3D), + Collections.singletonList(1L) + }, + new Object[]{ + Arrays.asList("a", "b", "c"), + Arrays.asList(2L, 3L), + Arrays.asList(3.3D, 4.4D, 5.5D), + null + }, new Object[]{null, Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), null}, new Object[]{null, null, null, null}, - new Object[]{Arrays.asList("d", "e"), Arrays.asList(1L, 4L), Arrays.asList(2.2D, 3.3D, 4.0D), Arrays.asList(1L, 2L)}, + new Object[]{ + Arrays.asList("d", "e"), + Arrays.asList(1L, 4L), + Arrays.asList(2.2D, 3.3D, 4.0D), + Arrays.asList(1L, 2L) + }, new Object[]{Arrays.asList("a", "b"), null, null, null}, - new Object[]{Arrays.asList("a", "b"), Arrays.asList(1L, 2L, 3L), Arrays.asList(1.1D, 2.2D, 3.3D), Arrays.asList(2L, 3L)}, - new Object[]{Arrays.asList("b", "c"), Arrays.asList(1L, 2L, 3L, 4L), Arrays.asList(1.1D, 3.3D), Collections.singletonList(1L)}, + new Object[]{ + Arrays.asList("a", "b"), + Arrays.asList(1L, 2L, 3L), + Arrays.asList(1.1D, 2.2D, 3.3D), + Arrays.asList(2L, 3L) + }, + new Object[]{ + Arrays.asList("b", "c"), + Arrays.asList(1L, 2L, 3L, 4L), + Arrays.asList(1.1D, 3.3D), + Collections.singletonList(1L) + }, new Object[]{Arrays.asList("a", "b", "c"), Arrays.asList(2L, 3L), Arrays.asList(3.3D, 4.4D, 5.5D), null} ) @@ -1014,6 +1049,7 @@ public void testUnnestRootSingleTypeArrayLongNulls() ) .run(); } + @Test public void testUnnestRootSingleTypeArrayStringNulls() { @@ -1183,6 +1219,58 @@ public void testGroupByRootSingleTypeArrayLong() .run(); } + @Test + public void testGroupByRootSingleTypeArrayLongFilteredArrayEquality() + { + cannotVectorize(); + testBuilder() + .sql( + "SELECT " + + "arrayLong, " + + "SUM(cnt) " + + "FROM druid.arrays WHERE arrayLong = ARRAY[1, 2, 3] GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter( + // don't use static functions since context flag indicates to always use equality filter + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{1L, 2L, 3L}, + null, + null + ) + ) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLong", "d0", ColumnType.LONG_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{Arrays.asList(1L, 2L, 3L), 4L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayLong", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ) + .run(); + } + @Test public void testGroupByRootSingleTypeArrayLongNulls() { @@ -1232,6 +1320,59 @@ public void testGroupByRootSingleTypeArrayLongNulls() .run(); } + @Test + public void testGroupByRootSingleTypeArrayLongNullsFilteredArrayEquality() + { + if (NullHandling.replaceWithDefault()) { + // this fails in default value mode because it relies on equality filter and null filter to behave correctly + return; + } + cannotVectorize(); + testBuilder() + .sql( + "SELECT " + + "arrayLongNulls, " + + "SUM(cnt) " + + "FROM druid.arrays WHERE arrayLongNulls = ARRAY[null, 2, 9] OR arrayLongNulls IS NULL GROUP BY 1" + ) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .expectedQueries( + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ARRAYS) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimFilter( + or( + equality("arrayLongNulls", new Object[]{null, 2L, 9L}, ColumnType.LONG_ARRAY), + isNull("arrayLongNulls") + ) + ) + .setDimensions( + dimensions( + new DefaultDimensionSpec("arrayLongNulls", "d0", ColumnType.LONG_ARRAY) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .build() + ) + ) + .expectedResults( + ImmutableList.of( + new Object[]{null, 3L}, + new Object[]{Arrays.asList(null, 2L, 9L), 2L} + ) + ) + .expectedSignature( + RowSignature.builder() + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("EXPR$1", ColumnType.LONG) + .build() + ) + .run(); + } + @Test public void testGroupByRootSingleTypeArrayLongNullsUnnest() { @@ -1311,10 +1452,18 @@ public void testGroupByRootSingleTypeArrayLongNullsFiltered() ) ) .setVirtualColumns( - new ExpressionVirtualColumn("v0", "array_length(\"arrayLongNulls\")", ColumnType.LONG, queryFramework().macroTable()) + new ExpressionVirtualColumn( + "v0", + "array_length(\"arrayLongNulls\")", + ColumnType.LONG, + queryFramework().macroTable() + ) ) .setDimFilter( - new ExpressionDimFilter("array_contains(\"arrayLongNulls\",1)", queryFramework().macroTable()) + new ExpressionDimFilter( + "array_contains(\"arrayLongNulls\",1)", + queryFramework().macroTable() + ) ) .setAggregatorSpecs( aggregators( @@ -1508,7 +1657,11 @@ public void testGroupByRootSingleTypeArrayStringNullsUnnest() .setDataSource( UnnestDataSource.create( TableDataSource.create(DATA_SOURCE_ARRAYS), - expressionVirtualColumn("j0.unnest", "\"arrayStringNulls\"", ColumnType.STRING_ARRAY), + expressionVirtualColumn( + "j0.unnest", + "\"arrayStringNulls\"", + ColumnType.STRING_ARRAY + ), null ) ) @@ -1568,10 +1721,18 @@ public void testGroupByRootSingleTypeArrayStringNullsFiltered() ) ) .setVirtualColumns( - new ExpressionVirtualColumn("v0", "array_length(\"arrayStringNulls\")", ColumnType.LONG, queryFramework().macroTable()) + new ExpressionVirtualColumn( + "v0", + "array_length(\"arrayStringNulls\")", + ColumnType.LONG, + queryFramework().macroTable() + ) ) .setDimFilter( - new ExpressionDimFilter("array_contains(\"arrayStringNulls\",'b')", queryFramework().macroTable()) + new ExpressionDimFilter( + "array_contains(\"arrayStringNulls\",'b')", + queryFramework().macroTable() + ) ) .setAggregatorSpecs( aggregators( @@ -1714,7 +1875,11 @@ public void testGroupByRootSingleTypeArrayDoubleNullsUnnest() .setDataSource( UnnestDataSource.create( TableDataSource.create(DATA_SOURCE_ARRAYS), - expressionVirtualColumn("j0.unnest", "\"arrayDoubleNulls\"", ColumnType.DOUBLE_ARRAY), + expressionVirtualColumn( + "j0.unnest", + "\"arrayDoubleNulls\"", + ColumnType.DOUBLE_ARRAY + ), null ) ) @@ -1775,10 +1940,18 @@ public void testGroupByRootSingleTypeArrayDoubleNullsFiltered() ) ) .setVirtualColumns( - new ExpressionVirtualColumn("v0", "array_length(\"arrayDoubleNulls\")", ColumnType.LONG, queryFramework().macroTable()) + new ExpressionVirtualColumn( + "v0", + "array_length(\"arrayDoubleNulls\")", + ColumnType.LONG, + queryFramework().macroTable() + ) ) .setDimFilter( - new ExpressionDimFilter("array_contains(\"arrayDoubleNulls\",2.2)", queryFramework().macroTable()) + new ExpressionDimFilter( + "array_contains(\"arrayDoubleNulls\",2.2)", + queryFramework().macroTable() + ) ) .setAggregatorSpecs( aggregators( @@ -1882,7 +2055,7 @@ public void testGroupByRootSingleTypeArrayLongElementFiltered() .setVirtualColumns( new NestedFieldVirtualColumn("arrayLong", "$[1]", "v0", ColumnType.LONG) ) - .setDimFilter(new SelectorDimFilter("v0", "2", null)) + .setDimFilter(equality("v0", 2L, ColumnType.LONG)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) .build() @@ -2022,7 +2195,7 @@ public void testGroupByRootSingleTypeArrayStringElementFiltered() .setVirtualColumns( new NestedFieldVirtualColumn("arrayStringNulls", "$[1]", "v0", ColumnType.STRING) ) - .setDimFilter(new SelectorDimFilter("v0", "b", null)) + .setDimFilter(equality("v0", "b", ColumnType.STRING)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) .build() @@ -2099,7 +2272,7 @@ public void testGroupByRootSingleTypeArrayDoubleElementFiltered() + "WHERE JSON_VALUE(arrayDoubleNulls, '$[2]' RETURNING DOUBLE) = 5.5" + "GROUP BY 1" ) - .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .queryContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY) .expectedQueries( ImmutableList.of( GroupByQuery.builder() @@ -2111,12 +2284,15 @@ public void testGroupByRootSingleTypeArrayDoubleElementFiltered() new DefaultDimensionSpec("v0", "d0", ColumnType.DOUBLE) ) ) - .setDimFilter(new SelectorDimFilter("v0", "5.5", null)) + .setDimFilter( + // dont use static function since context flag indicates to always use equality + new EqualityFilter("v0", ColumnType.DOUBLE, 5.5, null, null) + ) .setVirtualColumns( new NestedFieldVirtualColumn("arrayDoubleNulls", "$[2]", "v0", ColumnType.DOUBLE) ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) - .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY) .build() ) ) @@ -2196,7 +2372,7 @@ public void testGroupByPathSelectorFilter() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(selector("v0", "100", null)) + .setDimFilter(equality("v0", "100", ColumnType.STRING)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2359,7 +2535,7 @@ public void testGroupByPathSelectorFilterLong() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(selector("v0", "100", null)) + .setDimFilter(equality("v0", 100L, ColumnType.LONG)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2399,7 +2575,7 @@ public void testGroupByPathSelectorFilterDouble() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(selector("v0", "2.02", null)) + .setDimFilter(equality("v0", 2.02, ColumnType.DOUBLE)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2439,7 +2615,7 @@ public void testGroupByPathSelectorFilterString() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(selector("v0", "400", null)) + .setDimFilter(equality("v0", "400", ColumnType.STRING)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2479,7 +2655,7 @@ public void testGroupByPathSelectorFilterVariant() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(selector("v0", "1", null)) + .setDimFilter(equality("v0", 1L, ColumnType.LONG)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2514,7 +2690,7 @@ public void testGroupByPathSelectorFilterVariant2() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(selector("v0", "1", null)) + .setDimFilter(equality("v0", "1", ColumnType.STRING)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2530,6 +2706,92 @@ public void testGroupByPathSelectorFilterVariant2() ); } + @Test + public void testGroupByPathSelectorFilterVariant2Int() + { + testQuery( + "SELECT " + + "JSON_VALUE(nest, '$.x'), " + + "SUM(cnt) " + + "FROM druid.nested WHERE JSON_VALUE(nest, '$.mixed2') = 1 GROUP BY 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + new NestedFieldVirtualColumn("nest", "$.mixed2", "v0", ColumnType.LONG), + new NestedFieldVirtualColumn("nest", "$.x", "v1", ColumnType.STRING) + ) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v1", "d0") + ) + ) + .setDimFilter(equality("v0", 1L, ColumnType.LONG)) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + // this is a bit wonky, we get extra matches for numeric 1 matcher because the virtual column is defined + // as long typed, which makes a long processor which will convert the 1.1 to a 1L + new Object[]{"100", 2L}, + new Object[]{"200", 1L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + + @Test + public void testGroupByPathSelectorFilterVariant2BothTypesMatcher() + { + testQuery( + "SELECT " + + "JSON_VALUE(nest, '$.x'), " + + "SUM(cnt) " + + "FROM druid.nested WHERE JSON_VALUE(nest, '$.mixed2') = '1' OR JSON_VALUE(nest, '$.mixed2') = 1 GROUP BY 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + new NestedFieldVirtualColumn("nest", "$.mixed2", "v0", ColumnType.STRING), + new NestedFieldVirtualColumn("nest", "$.mixed2", "v1", ColumnType.LONG), + new NestedFieldVirtualColumn("nest", "$.x", "v2", ColumnType.STRING) + ) + .setDimensions( + dimensions( + new DefaultDimensionSpec("v2", "d0") + ) + ) + .setDimFilter( + or( + equality("v0", "1", ColumnType.STRING), + equality("v1", 1L, ColumnType.LONG) + ) + ) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + // this is a bit wonky, we get 2 matches for numeric 1 matcher because the virtual column is defined as + // long typed, which makes a long processor which will convert the 1.1 to a 1L + new Object[]{"100", 2L}, + new Object[]{"200", 1L} + ), + RowSignature.builder() + .add("EXPR$0", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + @Test public void testGroupByPathSelectorFilterVariant3() { @@ -2590,7 +2852,7 @@ public void testGroupByPathSelectorFilterNonExistent() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(selector("v0", "no way", null)) + .setDimFilter(equality("v0", "no way", ColumnType.STRING)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2625,7 +2887,7 @@ public void testGroupByPathSelectorFilterNull() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(not(selector("v0", null, null))) + .setDimFilter(notNull("v0")) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2662,7 +2924,7 @@ public void testGroupByPathBoundFilterLong() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(bound("v0", "100", "300", false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, "100", "300", false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2699,7 +2961,7 @@ public void testGroupByPathBoundFilterLongNoUpper() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(bound("v0", "100", null, false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, "100", null, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2736,7 +2998,7 @@ public void testGroupByPathBoundFilterLongNoLower() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(bound("v0", null, "100", false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, null, "100", false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2773,7 +3035,7 @@ public void testGroupByPathBoundFilterLongNumeric() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", "100", "300", false, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.LONG, 100L, 300L, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2811,7 +3073,7 @@ public void testGroupByPathBoundFilterLongNoUpperNumeric() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", "100", null, false, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.LONG, 100L, null, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2848,7 +3110,7 @@ public void testGroupByPathNumericBoundFilterLongNoUpperNumeric() new DefaultDimensionSpec("v0", "d0", ColumnType.LONG) ) ) - .setDimFilter(bound("v0", "100", null, false, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.LONG, 100L, null, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2886,13 +3148,17 @@ public void testGroupByPathBoundFilterLongNoLowerNumeric() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", null, "100", false, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.LONG, null, 100L, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of( - new Object[]{NullHandling.defaultStringValue(), 4L}, + NullHandling.replaceWithDefault() + ? ImmutableList.of( + new Object[]{"", 4L}, + new Object[]{"100", 2L} + ) + : ImmutableList.of( new Object[]{"100", 2L} ), RowSignature.builder() @@ -2923,7 +3189,7 @@ public void testGroupByPathBoundFilterDouble() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(bound("v0", "1.01", "3.03", false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, "1.01", "3.03", false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2960,7 +3226,7 @@ public void testGroupByPathBoundFilterDoubleNoUpper() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(bound("v0", "1.01", null, false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, "1.01", null, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -2997,7 +3263,7 @@ public void testGroupByPathBoundFilterDoubleNoLower() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(bound("v0", null, "2.02", false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, null, "2.02", false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3034,7 +3300,7 @@ public void testGroupByPathBoundDoubleFilterNumeric() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", "2.0", "3.5", false, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.DOUBLE, 2.0, 3.5, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3072,7 +3338,7 @@ public void testGroupByPathBoundFilterDoubleNoUpperNumeric() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", "1.0", null, false, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.DOUBLE, 1.0, null, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3110,13 +3376,17 @@ public void testGroupByPathBoundFilterDoubleNoLowerNumeric() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", null, "2.02", false, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.DOUBLE, null, 2.02, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of( - new Object[]{NullHandling.defaultStringValue(), 4L}, + NullHandling.replaceWithDefault() + ? ImmutableList.of( + new Object[]{"", 4L}, + new Object[]{"2.02", 2L} + ) + : ImmutableList.of( new Object[]{"2.02", 2L} ), RowSignature.builder() @@ -3147,7 +3417,7 @@ public void testGroupByPathBoundFilterString() new DefaultDimensionSpec("v0", "d0") ) ) - .setDimFilter(bound("v0", "100", "300", false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, "100", "300", false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3185,7 +3455,7 @@ public void testGroupByPathBoundFilterStringNoUpper() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", "400", null, false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, "400", null, false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3223,13 +3493,17 @@ public void testGroupByPathBoundFilterStringNoLower() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(bound("v0", null, "400", false, false, null, StringComparators.LEXICOGRAPHIC)) + .setDimFilter(range("v0", ColumnType.STRING, null, "400", false, false)) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of( - new Object[]{NullHandling.defaultStringValue(), 4L}, + NullHandling.replaceWithDefault() + ? ImmutableList.of( + new Object[]{"", 4L}, + new Object[]{"100", 2L} + ) + : ImmutableList.of( new Object[]{"100", 2L} ), RowSignature.builder() @@ -3408,7 +3682,12 @@ public void testGroupByPathInFilter() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(new InDimFilter("v0", ImmutableSet.of("100", "200"))) + .setDimFilter( + NullHandling.replaceWithDefault() + ? in("v0", ImmutableSet.of("100", "200"), null) + : or(equality("v0", 100L, ColumnType.LONG), equality("v0", 200L, ColumnType.LONG) + ) + ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3446,7 +3725,14 @@ public void testGroupByPathInFilterDouble() new DefaultDimensionSpec("v1", "d0") ) ) - .setDimFilter(new InDimFilter("v0", ImmutableSet.of("2.02", "3.03"))) + .setDimFilter( + NullHandling.replaceWithDefault() + ? in("v0", ImmutableSet.of("2.02", "3.03"), null) + : or( + equality("v0", 2.02, ColumnType.DOUBLE), + equality("v0", 3.03, ColumnType.DOUBLE) + ) + ) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -3586,7 +3872,7 @@ public void testSumPathFilteredAggDouble() aggregators( new FilteredAggregatorFactory( new DoubleSumAggregatorFactory("a0", "v1"), - selector("v0", "2.02", null) + equality("v0", 2.02, ColumnType.DOUBLE) ) ) ) @@ -3623,7 +3909,7 @@ public void testSumPathFilteredAggString() aggregators( new FilteredAggregatorFactory( new DoubleSumAggregatorFactory("a0", "v1"), - selector("v0", "300", null) + equality("v0", "300", ColumnType.STRING) ) ) ) @@ -3686,12 +3972,11 @@ public void testSumPathMixedFilteredAggLong() new NestedFieldVirtualColumn("nest", "$.mixed", "v0", ColumnType.LONG), new NestedFieldVirtualColumn("nest", "$.mixed", "v1", ColumnType.DOUBLE) ) - .aggregators( aggregators( new FilteredAggregatorFactory( new DoubleSumAggregatorFactory("a0", "v1"), - selector("v0", "1", null) + equality("v0", 1L, ColumnType.LONG) ) ) ) @@ -3727,7 +4012,7 @@ public void testSumPathMixedFilteredAggDouble() aggregators( new FilteredAggregatorFactory( new DoubleSumAggregatorFactory("a0", "v0"), - selector("v0", "1.1", null) + equality("v0", 1.1, ColumnType.DOUBLE) ) ) ) @@ -4623,7 +4908,7 @@ public void testGroupByPathSelectorFilterVariantNull() new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) - .setDimFilter(selector("v0", null, null)) + .setDimFilter(isNull("v0")) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4660,7 +4945,7 @@ public void testSelectPathSelectorFilterVariantNull() .columns( "v0", "v1" ) - .filters(selector("v0", null, null)) + .filters(isNull("v0")) .context(QUERY_CONTEXT_DEFAULT) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) @@ -4705,7 +4990,7 @@ public void testGroupByPathSelectorFilterVariantNotNull() new DefaultDimensionSpec("v0", "d1", ColumnType.LONG) ) ) - .setDimFilter(not(selector("v0", null, null))) + .setDimFilter(notNull("v0")) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4740,7 +5025,7 @@ public void testGroupByRegularLongLongMixed1FilterNotNull() ) ) .setVirtualColumns(new NestedFieldVirtualColumn("long", "$", "v0", ColumnType.LONG)) - .setDimFilter(not(selector("v0", null, null))) + .setDimFilter(notNull("v0")) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4778,7 +5063,7 @@ public void testGroupByRootSingleTypeStringMixed1SparseNotNull() ) ) .setVirtualColumns(new NestedFieldVirtualColumn("string_sparse", "$", "v0", ColumnType.LONG)) - .setDimFilter(not(selector("v0", null, null))) + .setDimFilter(notNull("v0")) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4808,7 +5093,7 @@ public void testScanStringNotNullCast() .virtualColumns( expressionVirtualColumn("v0", "CAST(\"string_sparse\", 'LONG')", ColumnType.LONG) ) - .filters(not(selector("v0", null, null))) + .filters(notNull("v0")) .columns("v0") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) @@ -4859,8 +5144,12 @@ public void testGroupByRootSingleTypeStringMixed1SparseNotNullCast2() new DefaultDimensionSpec("string_sparse", "d0", ColumnType.LONG) ) ) - .setVirtualColumns(expressionVirtualColumn("v0", "CAST(\"string_sparse\", 'LONG')", ColumnType.LONG)) - .setDimFilter(not(selector("v0", null, null))) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "CAST(\"string_sparse\", 'LONG')", + ColumnType.LONG + )) + .setDimFilter(notNull("v0")) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4906,7 +5195,11 @@ public void testGroupByRootSingleTypeArrayLongNullsAsMvd() new DefaultDimensionSpec("v0", "d0", ColumnType.STRING) ) ) - .setVirtualColumns(expressionVirtualColumn("v0", "array_to_mv(\"arrayLongNulls\")", ColumnType.STRING)) + .setVirtualColumns(expressionVirtualColumn( + "v0", + "array_to_mv(\"arrayLongNulls\")", + ColumnType.STRING + )) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY) .build() @@ -4965,10 +5258,17 @@ public void testGroupByRootSingleTypeArrayLongNullsAsMvdFiltered() ) .setVirtualColumns( expressionVirtualColumn("v0", "array_to_mv(\"arrayLongNulls\")", ColumnType.STRING), - expressionVirtualColumn("v1", "array_length(array_to_mv(\"arrayLongNulls\"))", ColumnType.LONG) + expressionVirtualColumn( + "v1", + "array_length(array_to_mv(\"arrayLongNulls\"))", + ColumnType.LONG + ) ) .setDimFilter( - new ExpressionDimFilter("array_contains(array_to_mv(\"arrayLongNulls\"),'1')", queryFramework().macroTable()) + new ExpressionDimFilter( + "array_contains(array_to_mv(\"arrayLongNulls\"),'1')", + queryFramework().macroTable() + ) ) .setAggregatorSpecs( aggregators( @@ -5145,10 +5445,18 @@ public void testGroupByRootSingleTypeArrayStringNullsFilteredAsMvd() ) .setVirtualColumns( expressionVirtualColumn("v0", "array_to_mv(\"arrayStringNulls\")", ColumnType.STRING), - new ExpressionVirtualColumn("v1", "array_length(array_to_mv(\"arrayStringNulls\"))", ColumnType.LONG, queryFramework().macroTable()) + new ExpressionVirtualColumn( + "v1", + "array_length(array_to_mv(\"arrayStringNulls\"))", + ColumnType.LONG, + queryFramework().macroTable() + ) ) .setDimFilter( - new ExpressionDimFilter("array_contains(array_to_mv(\"arrayStringNulls\"),'b')", queryFramework().macroTable()) + new ExpressionDimFilter( + "array_contains(array_to_mv(\"arrayStringNulls\"),'b')", + queryFramework().macroTable() + ) ) .setAggregatorSpecs( aggregators( @@ -5178,6 +5486,39 @@ public void testGroupByRootSingleTypeArrayStringNullsFilteredAsMvd() .run(); } + @Test + public void testGroupByAndFilterVariant() + { + testQuery( + "SELECT " + + "variant, " + + "SUM(cnt) " + + "FROM druid.all_auto WHERE variant = '1' GROUP BY 1", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(DATA_SOURCE_ALL) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions( + dimensions( + new DefaultDimensionSpec("variant", "d0") + ) + ) + .setDimFilter(equality("variant", "1", ColumnType.STRING)) + .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"1", 2L} + ), + RowSignature.builder() + .add("variant", ColumnType.STRING) + .add("EXPR$1", ColumnType.LONG) + .build() + ); + } + @Test public void testScanAllTypesAuto() { @@ -5230,22 +5571,512 @@ public void testScanAllTypesAuto() ), useDefault ? ImmutableList.of( - new Object[]{1672531200000L, "", 0L, 0.0D, "true", "51", "1", "[]", "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", null, "[\"a\",\"b\"]", null, "[2,3]", null, "[null]", null, "[\"true\",\"false\",\"true\"]", null, "[{\"x\":1},{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "", 2L, 0.0D, "false", "b", "\"b\"", "2", "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}", "[\"a\",\"b\",\"c\"]", "[null,\"b\"]", "[2,3]", null, "[3.3,4.4,5.5]", "[999.0,null,5.5]", "[null,null,2.2]", "[\"true\",\"true\"]", "[null,[null],[]]", "[{\"x\":3},{\"x\":4}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "a", 1L, 1.0D, "true", "1", "1", "1", "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"a\",\"b\"]", "[\"a\",\"b\"]", "[1,2,3]", "[1,null,3]", "[1.1,2.2,3.3]", "[1.1,2.2,null]", "[\"a\",\"1\",\"2.2\"]", "[\"true\",\"false\",\"true\"]", "[[1,2,null],[3,4]]", "[{\"x\":1},{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "b", 4L, 3.3D, "true", "4", "{}", "4", "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"d\",\"e\"]", "[\"b\",\"b\"]", "[1,4]", "[1]", "[2.2,3.3,4.0]", null, "[\"a\",\"b\",\"c\"]", "[null,\"false\",\"true\"]", "[[1,2],[3,4],[5,6,7]]", "[{\"x\":null},{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "c", 0L, 4.4D, "true", "hello", "{}", "[]", "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, null, "[1,2,3]", "[]", "[1.1,2.2,3.3]", null, null, "[\"false\"]", null, "[{\"x\":1000},{\"y\":2000}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "d", 5L, 5.9D, "false", "", "\"a\"", "6", "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", null, "[\"a\",\"b\"]", null, null, "[null,2,9]", null, "[999.0,5.5,null]", "[\"a\",\"1\",\"2.2\"]", "[]", "[[1],[1,2,null]]", "[{\"a\":1},{\"b\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "null", 3L, 2.0D, "", "3.0", "3.3", "3", "{\"a\":300}", "{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}", "[\"b\",\"c\"]", "[\"d\",null,\"b\"]", "[1,2,3,4]", "[1,2,3]", "[1.1,3.3]", "[null,2.2,null]", "[1,null,1]", "[\"true\",null,\"true\"]", "[[1],null,[1,2,3]]", "[null,{\"x\":2}]", "", "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L} + new Object[]{ + 1672531200000L, + "", + 0L, + 0.0D, + "true", + "51", + "1", + "[]", + "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", + "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", + null, + "[\"a\",\"b\"]", + null, + "[2,3]", + null, + "[null]", + null, + "[\"true\",\"false\",\"true\"]", + null, + "[{\"x\":1},{\"x\":2}]", + "", + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "", + 2L, + 0.0D, + "false", + "b", + "\"b\"", + "2", + "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", + "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}", + "[\"a\",\"b\",\"c\"]", + "[null,\"b\"]", + "[2,3]", + null, + "[3.3,4.4,5.5]", + "[999.0,null,5.5]", + "[null,null,2.2]", + "[\"true\",\"true\"]", + "[null,[null],[]]", + "[{\"x\":3},{\"x\":4}]", + "", + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "a", + 1L, + 1.0D, + "true", + "1", + "1", + "1", + "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", + "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", + "[\"a\",\"b\"]", + "[\"a\",\"b\"]", + "[1,2,3]", + "[1,null,3]", + "[1.1,2.2,3.3]", + "[1.1,2.2,null]", + "[\"a\",\"1\",\"2.2\"]", + "[\"true\",\"false\",\"true\"]", + "[[1,2,null],[3,4]]", + "[{\"x\":1},{\"x\":2}]", + "", + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "b", + 4L, + 3.3D, + "true", + "1", + "{}", + "4", + "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", + "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", + "[\"d\",\"e\"]", + "[\"b\",\"b\"]", + "[1,4]", + "[1]", + "[2.2,3.3,4.0]", + null, + "[\"a\",\"b\",\"c\"]", + "[null,\"false\",\"true\"]", + "[[1,2],[3,4],[5,6,7]]", + "[{\"x\":null},{\"x\":2}]", + "", + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "c", + 0L, + 4.4D, + "true", + "hello", + "{}", + "[]", + "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", + "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", + null, + null, + "[1,2,3]", + "[]", + "[1.1,2.2,3.3]", + null, + null, + "[\"false\"]", + null, + "[{\"x\":1000},{\"y\":2000}]", + "", + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "d", + 5L, + 5.9D, + "false", + "", + "\"a\"", + "6", + "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", + null, + "[\"a\",\"b\"]", + null, + null, + "[null,2,9]", + null, + "[999.0,5.5,null]", + "[\"a\",\"1\",\"2.2\"]", + "[]", + "[[1],[1,2,null]]", + "[{\"a\":1},{\"b\":2}]", + "", + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "null", + 3L, + 2.0D, + "", + "3.0", + "3.3", + "3", + "{\"a\":300}", + "{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}", + "[\"b\",\"c\"]", + "[\"d\",null,\"b\"]", + "[1,2,3,4]", + "[1,2,3]", + "[1.1,3.3]", + "[null,2.2,null]", + "[1,null,1]", + "[\"true\",null,\"true\"]", + "[[1],null,[1,2,3]]", + "[null,{\"x\":2}]", + "", + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + } ) : ImmutableList.of( - new Object[]{1672531200000L, null, null, null, "true", "51", "1", "[]", "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", null, "[\"a\",\"b\"]", null, "[2,3]", null, "[null]", null, "[\"true\",\"false\",\"true\"]", null, "[{\"x\":1},{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "", 2L, null, "false", "b", "\"b\"", "2", "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}", "[\"a\",\"b\",\"c\"]", "[null,\"b\"]", "[2,3]", null, "[3.3,4.4,5.5]", "[999.0,null,5.5]", "[null,null,2.2]", "[\"true\",\"true\"]", "[null,[null],[]]", "[{\"x\":3},{\"x\":4}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "a", 1L, 1.0D, "true", "1", "1", "1", "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"a\",\"b\"]", "[\"a\",\"b\"]", "[1,2,3]", "[1,null,3]", "[1.1,2.2,3.3]", "[1.1,2.2,null]", "[\"a\",\"1\",\"2.2\"]", "[\"true\",\"false\",\"true\"]", "[[1,2,null],[3,4]]", "[{\"x\":1},{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "b", 4L, 3.3D, "true", "4", "{}", "4", "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"d\",\"e\"]", "[\"b\",\"b\"]", "[1,4]", "[1]", "[2.2,3.3,4.0]", null, "[\"a\",\"b\",\"c\"]", "[null,\"false\",\"true\"]", "[[1,2],[3,4],[5,6,7]]", "[{\"x\":null},{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "c", null, 4.4D, "true", "hello", "{}", "[]", "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, null, "[1,2,3]", "[]", "[1.1,2.2,3.3]", null, null, "[\"false\"]", null, "[{\"x\":1000},{\"y\":2000}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "d", 5L, 5.9D, "false", null, "\"a\"", "6", "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", null, "[\"a\",\"b\"]", null, null, "[null,2,9]", null, "[999.0,5.5,null]", "[\"a\",\"1\",\"2.2\"]", "[]", "[[1],[1,2,null]]", "[{\"a\":1},{\"b\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L}, - new Object[]{1672531200000L, "null", 3L, 2.0D, null, "3.0", "3.3", "3", "{\"a\":300}", "{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}", "[\"b\",\"c\"]", "[\"d\",null,\"b\"]", "[1,2,3,4]", "[1,2,3]", "[1.1,3.3]", "[null,2.2,null]", "[1,null,1]", "[\"true\",null,\"true\"]", "[[1],null,[1,2,3]]", "[null,{\"x\":2}]", null, "hello", 1234L, 1.234D, "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", "[\"a\",\"b\",\"c\"]", "[1,2,3]", "[1.1,2.2,3.3]", "[]", "{}", "[null,null]", "[{},{},{}]", "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", 1L} + new Object[]{ + 1672531200000L, + null, + null, + null, + "true", + "51", + "1", + "[]", + "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", + "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", + null, + "[\"a\",\"b\"]", + null, + "[2,3]", + null, + "[null]", + null, + "[\"true\",\"false\",\"true\"]", + null, + "[{\"x\":1},{\"x\":2}]", + null, + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "", + 2L, + null, + "false", + "b", + "\"b\"", + "2", + "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", + "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}", + "[\"a\",\"b\",\"c\"]", + "[null,\"b\"]", + "[2,3]", + null, + "[3.3,4.4,5.5]", + "[999.0,null,5.5]", + "[null,null,2.2]", + "[\"true\",\"true\"]", + "[null,[null],[]]", + "[{\"x\":3},{\"x\":4}]", + null, + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "a", + 1L, + 1.0D, + "true", + "1", + "1", + "1", + "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", + "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", + "[\"a\",\"b\"]", + "[\"a\",\"b\"]", + "[1,2,3]", + "[1,null,3]", + "[1.1,2.2,3.3]", + "[1.1,2.2,null]", + "[\"a\",\"1\",\"2.2\"]", + "[\"true\",\"false\",\"true\"]", + "[[1,2,null],[3,4]]", + "[{\"x\":1},{\"x\":2}]", + null, + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "b", + 4L, + 3.3D, + "true", + "1", + "{}", + "4", + "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", + "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", + "[\"d\",\"e\"]", + "[\"b\",\"b\"]", + "[1,4]", + "[1]", + "[2.2,3.3,4.0]", + null, + "[\"a\",\"b\",\"c\"]", + "[null,\"false\",\"true\"]", + "[[1,2],[3,4],[5,6,7]]", + "[{\"x\":null},{\"x\":2}]", + null, + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "c", + null, + 4.4D, + "true", + "hello", + "{}", + "[]", + "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", + "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", + null, + null, + "[1,2,3]", + "[]", + "[1.1,2.2,3.3]", + null, + null, + "[\"false\"]", + null, + "[{\"x\":1000},{\"y\":2000}]", + null, + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "d", + 5L, + 5.9D, + "false", + null, + "\"a\"", + "6", + "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", + null, + "[\"a\",\"b\"]", + null, + null, + "[null,2,9]", + null, + "[999.0,5.5,null]", + "[\"a\",\"1\",\"2.2\"]", + "[]", + "[[1],[1,2,null]]", + "[{\"a\":1},{\"b\":2}]", + null, + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + }, + new Object[]{ + 1672531200000L, + "null", + 3L, + 2.0D, + null, + "3.0", + "3.3", + "3", + "{\"a\":300}", + "{\"x\":4,\"y\":[{\"l\":[],\"m\":100,\"n\":3},{\"l\":[\"a\"]},{\"l\":[\"b\"],\"n\":[]}],\"z\":{\"a\":[],\"b\":true}}", + "[\"b\",\"c\"]", + "[\"d\",null,\"b\"]", + "[1,2,3,4]", + "[1,2,3]", + "[1.1,3.3]", + "[null,2.2,null]", + "[1,null,1]", + "[\"true\",null,\"true\"]", + "[[1],null,[1,2,3]]", + "[null,{\"x\":2}]", + null, + "hello", + 1234L, + 1.234D, + "{\"x\":1,\"y\":\"hello\",\"z\":{\"a\":1.1,\"b\":1234,\"c\":[\"a\",\"b\",\"c\"]}}", + "[\"a\",\"b\",\"c\"]", + "[1,2,3]", + "[1.1,2.2,3.3]", + "[]", + "{}", + "[null,null]", + "[{},{},{}]", + "[{\"a\":\"b\",\"x\":1,\"y\":1.3}]", + 1L + } ), RowSignature.builder() .add("__time", ColumnType.LONG) @@ -5285,4 +6116,81 @@ public void testScanAllTypesAuto() .build() ); } + + @Test + public void testFilterJsonIsNotNull() + { + testQuery( + "SELECT nest\n" + + "FROM druid.nested WHERE nest IS NOT NULL", + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("nest") + .filters(notNull("nest")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() + ), + NullHandling.replaceWithDefault() + ? ImmutableList.of() + : ImmutableList.of( + new Object[]{"{\"x\":100,\"y\":2.02,\"z\":\"300\",\"mixed\":1,\"mixed2\":\"1\"}"}, + new Object[]{"{\"x\":200,\"y\":3.03,\"z\":\"abcdef\",\"mixed\":1.1,\"mixed2\":1}"}, + new Object[]{"{\"x\":100,\"y\":2.02,\"z\":\"400\",\"mixed2\":1.1}"} + ), + RowSignature.builder() + .add("nest", ColumnType.NESTED_DATA) + .build() + + ); + } + + @Test + public void testFilterJsonIsNull() + { + testQuery( + "SELECT nest, nester\n" + + "FROM druid.nested WHERE nest IS NULL", + ImmutableList.of( + Druids.newScanQueryBuilder() + .dataSource(DATA_SOURCE) + .intervals(querySegmentSpec(Filtration.eternity())) + .columns("nest", "nester") + .filters(isNull("nest")) + .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) + .legacy(false) + .build() + ), + // selector filter is wrong + NullHandling.replaceWithDefault() + ? ImmutableList.of( + new Object[]{ + "{\"x\":100,\"y\":2.02,\"z\":\"300\",\"mixed\":1,\"mixed2\":\"1\"}", + "{\"array\":[\"a\",\"b\"],\"n\":{\"x\":\"hello\"}}" + }, + new Object[]{null, "\"hello\""}, + new Object[]{"{\"x\":200,\"y\":3.03,\"z\":\"abcdef\",\"mixed\":1.1,\"mixed2\":1}", null}, + new Object[]{null, null}, + new Object[]{null, null}, + new Object[]{ + "{\"x\":100,\"y\":2.02,\"z\":\"400\",\"mixed2\":1.1}", + "{\"array\":[\"a\",\"b\"],\"n\":{\"x\":1}}" + }, + new Object[]{null, "2"} + ) + : ImmutableList.of( + new Object[]{null, "\"hello\""}, + new Object[]{null, null}, + new Object[]{null, null}, + new Object[]{null, "2"} + ), + RowSignature.builder() + .add("nest", ColumnType.NESTED_DATA) + .add("nester", ColumnType.NESTED_DATA) + .build() + + ); + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java index 2266b8d6b825..edda85260bda 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java @@ -122,7 +122,7 @@ public void testParametersInSelectAndFilter() ImmutableList.of(Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(numericSelector("dim2", "0", null)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .granularity(Granularities.ALL) .aggregators(aggregators( new CountAggregatorFactory("a0"), @@ -312,7 +312,7 @@ public void testParametersInStrangePlaces() .setAggregatorSpecs(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("dim2", "a", null)) + not(equality("dim2", "a", ColumnType.STRING)) ), new CountAggregatorFactory("a1") )) @@ -492,7 +492,9 @@ public void testDoubles() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - bound("cnt", "1.1", "100000001", true, true, null, StringComparators.NUMERIC) + NullHandling.replaceWithDefault() + ? bound("cnt", "1.1", "100000001", true, true, null, StringComparators.NUMERIC) + : range("cnt", ColumnType.DOUBLE, 1.1, 100000001.0, true, true) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -514,7 +516,12 @@ public void testDoubles() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - in("cnt", ImmutableList.of("1.0", "100000001"), null) + NullHandling.replaceWithDefault() + ? in("cnt", ImmutableList.of("1.0", "100000001"), null) + : or( + equality("cnt", 1.0, ColumnType.DOUBLE), + equality("cnt", 100000001.0, ColumnType.DOUBLE) + ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -541,7 +548,7 @@ public void testFloats() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - selector("cnt", "1.0", null) + equality("cnt", 1.0, ColumnType.DOUBLE) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -564,7 +571,7 @@ public void testLongs() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(bound("l1", "3", null, true, false, null, StringComparators.NUMERIC)) + .filters(range("l1", ColumnType.LONG, 3L, null, true, false)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -643,7 +650,7 @@ public void testWrongTypeParameter() .filters( and( bound("l1", "3", null, true, false, null, StringComparators.NUMERIC), - selector("f1", useDefault ? "0.0" : null, null) + selector("f1", "0.0", null) ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) 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 54c8b3607063..876ade4271ff 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 @@ -80,14 +80,13 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.RegexDimExtractionFn; import org.apache.druid.query.extraction.SubstringDimExtractionFn; -import org.apache.druid.query.filter.AndDimFilter; -import org.apache.druid.query.filter.BoundDimFilter; import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.LikeDimFilter; -import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.query.filter.RangeFilter; 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.GroupByQueryConfig; import org.apache.druid.query.groupby.orderby.DefaultLimitSpec; @@ -1391,13 +1390,13 @@ public void testFirstLatestAggregatorsSkipNulls() final DimFilter filter; if (useDefault) { - filter = not(selector("dim1", null, null)); + filter = notNull("dim1"); } else { filter = and( - not(selector("dim1", null, null)), - not(selector("l1", null, null)), - not(selector("d1", null, null)), - not(selector("f1", null, null)) + notNull("dim1"), + notNull("l1"), + notNull("d1"), + notNull("f1") ); } testQuery( @@ -1461,13 +1460,13 @@ public void testAnyAggregatorsSkipNullsWithFilter() { final DimFilter filter; if (useDefault) { - filter = not(selector("dim1", null, null)); + filter = notNull("dim1"); } else { filter = and( - not(selector("dim1", null, null)), - not(selector("l2", null, null)), - not(selector("d2", null, null)), - not(selector("f2", null, null)) + notNull("dim1"), + notNull("l2"), + notNull("d2"), + notNull("f2") ); } testQuery( @@ -2119,7 +2118,7 @@ public void testFilterOnFloat() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .filters(selector("m1", "1.0", null)) + .filters(equality("m1", 1.0, ColumnType.DOUBLE)) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2140,7 +2139,7 @@ public void testFilterOnDouble() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) - .filters(selector("m2", "1.0", null)) + .filters(equality("m2", 1.0, ColumnType.DOUBLE)) .context(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2161,7 +2160,13 @@ public void testHavingOnGrandTotal() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1"))) - .setHavingSpec(having(selector("a0", "21", null))) + .setHavingSpec( + having( + NullHandling.replaceWithDefault() + ? selector("a0", "21") + : equality("a0", 21.0, ColumnType.DOUBLE) + ) + ) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -2185,15 +2190,13 @@ public void testHavingOnDoubleSum() .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1"))) .setHavingSpec( having( - new BoundDimFilter( + range( "a0", - "1", + ColumnType.LONG, + 1L, null, true, - false, - false, - null, - StringComparators.NUMERIC + false ) ) ) @@ -2236,14 +2239,13 @@ public void testHavingOnApproximateCountDistinct() ) .setHavingSpec( having( - bound( + range( "a0", - "1", + ColumnType.LONG, + 1L, null, true, - false, - null, - StringComparators.NUMERIC + false ) ) ) @@ -2296,20 +2298,19 @@ public void testHavingOnExactCountDistinct() ? new CountAggregatorFactory("a0") : new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("d1", null, null)) + notNull("d1") ) ) ) .setHavingSpec( having( - bound( + range( "a0", - "1", + ColumnType.LONG, + 1L, null, true, - false, - null, - StringComparators.NUMERIC + false ) ) ) @@ -2407,13 +2408,13 @@ public void testExactCountDistinctWithFilter() new FilteredAggregatorFactory( new CountAggregatorFactory("_a0"), and( - not(selector("d0", null, null)), - selector("a1", "0", null) + notNull("d0"), + equality("a1", 0L, ColumnType.LONG) ) ), new FilteredAggregatorFactory( new LongMinAggregatorFactory("_a1", "a0"), - selector("a1", "3", null) + equality("a1", 3L, ColumnType.LONG) ) )) .setContext(QUERY_CONTEXT_DEFAULT) @@ -2439,15 +2440,13 @@ public void testHavingOnFloatSum() .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1"))) .setHavingSpec( having( - new BoundDimFilter( + range( "a0", - "1", + ColumnType.LONG, + 1L, null, true, - false, - false, - null, - StringComparators.NUMERIC + false ) ) ) @@ -2515,7 +2514,7 @@ public void testHavingOnRatio() .setAggregatorSpecs(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("dim2", "a", null)) + not(equality("dim2", "a", ColumnType.STRING)) ), new CountAggregatorFactory("a1") )) @@ -3278,7 +3277,13 @@ public void testPruneDeadAggregatorsThroughHaving() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setAggregatorSpecs(aggregators(new DoubleSumAggregatorFactory("a0", "m1"))) - .setHavingSpec(having(selector("a0", "21", null))) + .setHavingSpec( + having( + NullHandling.replaceWithDefault() + ? selector("a0", "21") + : equality("a0", 21.0, ColumnType.DOUBLE) + ) + ) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -3391,10 +3396,10 @@ public void testNullEmptyStringEquality() // (dim2 != 'a') component is unnecessary. .filters( or( - selector("dim2", "a", null), + equality("dim2", "a", ColumnType.STRING), and( - selector("dim2", null, null), - not(selector("dim2", "a", null)) + isNull("dim2"), + not(equality("dim2", "a", ColumnType.STRING)) ) ) ) @@ -3439,7 +3444,7 @@ public void testNullLongFilter() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(selector("l1", null, null)) + .filters(isNull("l1")) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3477,7 +3482,7 @@ public void testNullDoubleFilter() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(selector("d1", null, null)) + .filters(isNull("d1")) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3515,7 +3520,7 @@ public void testNullFloatFilter() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(selector("f1", null, null)) + .filters(isNull("f1")) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3710,7 +3715,7 @@ public void testLongPredicateFilterNulls() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(bound("l1", "3", null, true, false, null, StringComparators.NUMERIC)) + .filters(range("l1", ColumnType.LONG, 3L, null, true, false)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3731,7 +3736,7 @@ public void testDoublePredicateFilterNulls() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(bound("d1", "0", null, true, false, null, StringComparators.NUMERIC)) + .filters(range("d1", ColumnType.LONG, 0L, null, true, false)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3752,7 +3757,7 @@ public void testFloatPredicateFilterNulls() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(bound("f1", "0", null, true, false, null, StringComparators.NUMERIC)) + .filters(range("f1", ColumnType.LONG, 0L, null, true, false)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3794,7 +3799,7 @@ public void testEmptyStringEquality() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(selector("dim2", "", null)) + .filters(equality("dim2", "", ColumnType.STRING)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -3897,18 +3902,19 @@ public void testCoalesceColumnsFilter() ) ) .setDimFilter( - new OrDimFilter( - new AndDimFilter( - selector("dim1", "a", null), - selector("dim2", null, null) + or( + and( + equality("dim1", "a", ColumnType.STRING), + isNull("dim2") ), - new AndDimFilter( - selector("dim1", "abc", null), - selector("dim2", null, null) + and( + equality("dim1", "abc", ColumnType.STRING), + isNull("dim2") ), - new InDimFilter( + in( "dim2", - ImmutableSet.of("a", "abc") + ImmutableSet.of("a", "abc"), + null ) ) ) @@ -3924,6 +3930,59 @@ public void testCoalesceColumnsFilter() ); } + @Test + public void testCoalesceColumnsFilterWithEquality() + { + // Cannot vectorize due to virtual columns. + cannotVectorize(); + + // we can remove this test if PlannerContext.CTX_SQL_USE_BOUNDS_AND_SELECTORS ever defaults to false all the time + // since it otherwise is a duplicate of testCoalesceColumnsFilter + + testQuery( + "SELECT COALESCE(dim2, dim1), COUNT(*) FROM druid.foo WHERE COALESCE(dim2, dim1) IN ('a', 'abc') GROUP BY COALESCE(dim2, dim1)", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY, + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setVirtualColumns( + expressionVirtualColumn( + "v0", + "nvl(\"dim2\",\"dim1\")", + ColumnType.STRING + ) + ) + .setDimFilter( + or( + and( + new EqualityFilter("dim1", ColumnType.STRING, "a", null, null), + NullFilter.forColumn("dim2") + ), + and( + new EqualityFilter("dim1", ColumnType.STRING, "abc", null, null), + NullFilter.forColumn("dim2") + ), + in( + "dim2", + ImmutableSet.of("a", "abc"), + null + ) + ) + ) + .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.STRING))) + .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) + .setContext(QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY) + .build() + ), + ImmutableList.of( + new Object[]{"a", 2L}, + new Object[]{"abc", 2L} + ) + ); + } + @Test public void testCoalesceMoreColumns() { @@ -3989,7 +4048,7 @@ public void testColumnIsNull() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(selector("dim2", null, null)) + .filters(isNull("dim2")) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -4140,7 +4199,7 @@ public void testGroupByWithFilterMatchingNothing() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim1", "foobar", null)) + .filters(equality("dim1", "foobar", ColumnType.STRING)) .granularity(Granularities.ALL) .aggregators(aggregators( new CountAggregatorFactory("a0"), @@ -4186,7 +4245,7 @@ public void testGroupByWithFilterMatchingNothingWithGroupByLiteral() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim1", "foobar", null)) + .filters(equality("dim1", "foobar", ColumnType.STRING)) .granularity(Granularities.ALL) .aggregators(aggregators( new CountAggregatorFactory("a0"), @@ -4215,7 +4274,7 @@ public void testCountNonNullColumn() ? new CountAggregatorFactory("a0") : new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("cnt", null, null)) + notNull("cnt") ) ) ) @@ -4241,7 +4300,7 @@ public void testCountNullableColumn() .aggregators(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("dim2", null, null)) + notNull("dim2") ) )) .context(QUERY_CONTEXT_DEFAULT) @@ -4313,8 +4372,8 @@ public void testCountStarOnCommonTableExpression() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .filters(and( - selector("dim2", "a", null), - not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))) + equality("dim2", "a", ColumnType.STRING), + not(equality("dim1", "z", new SubstringDimExtractionFn(0, 1), ColumnType.STRING)) )) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -4337,8 +4396,8 @@ public void testCountStarOnView() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .filters(and( - selector("dim2", "a", null), - not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))) + equality("dim2", "a", ColumnType.STRING), + not(equality("dim1", "z", new SubstringDimExtractionFn(0, 1), ColumnType.STRING)) )) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -4361,8 +4420,8 @@ public void testConfusedView() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .filters(and( - selector("dim2", "a", null), - not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1))) + equality("dim2", "a", ColumnType.STRING), + not(equality("dim1", "z", new SubstringDimExtractionFn(0, 1), ColumnType.STRING)) )) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -4413,8 +4472,8 @@ public void testCountStarWithLongColumnFilters() .granularity(Granularities.ALL) .filters( or( - bound("cnt", "3", null, false, false, null, StringComparators.NUMERIC), - selector("cnt", "1", null) + range("cnt", ColumnType.LONG, 3L, null, false, false), + equality("cnt", 1L, ColumnType.LONG) ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -4427,6 +4486,35 @@ public void testCountStarWithLongColumnFilters() ); } + @Test + public void testCountStarWithLongColumnFiltersForceRange() + { + // we can remove this test if PlannerContext.CTX_SQL_USE_BOUNDS_AND_SELECTORS ever defaults to false all the time + // since it otherwise is a duplicate of testCountStarWithLongColumnFilters + testQuery( + "SELECT COUNT(*) FROM druid.foo WHERE cnt >= 3 OR cnt = 1", + QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY, + ImmutableList.of( + Druids.newTimeseriesQueryBuilder() + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .granularity(Granularities.ALL) + .filters( + or( + new RangeFilter("cnt", ColumnType.LONG, 3L, null, false, false, null, null), + new EqualityFilter("cnt", ColumnType.LONG, 1L, null, null) + ) + ) + .aggregators(aggregators(new CountAggregatorFactory("a0"))) + .context(QUERY_CONTEXT_NO_STRINGIFY_ARRAY_USE_EQUALITY) + .build() + ), + ImmutableList.of( + new Object[]{6L} + ) + ); + } + @Test public void testCountStarWithLongColumnFiltersOnFloatLiterals() { @@ -4439,7 +4527,9 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - bound("cnt", "1.1", "100000001.0", true, true, null, StringComparators.NUMERIC) + NullHandling.replaceWithDefault() + ? bound("cnt", "1.1", "100000001.0", true, true, null, StringComparators.NUMERIC) + : range("cnt", ColumnType.DOUBLE, 1.1, 100000001.0, true, true) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -4458,7 +4548,7 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - selector("cnt", "1.0", null) + equality("cnt", 1.0, ColumnType.DOUBLE) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -4477,7 +4567,9 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - selector("cnt", "100000001.0", null) + NullHandling.replaceWithDefault() + ? selector("cnt", "100000001.0") + : equality("cnt", 100000001.0, ColumnType.DOUBLE) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -4496,7 +4588,9 @@ public void testCountStarWithLongColumnFiltersOnFloatLiterals() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - in("cnt", ImmutableList.of("1.0", "100000001.0"), null) + NullHandling.replaceWithDefault() + ? in("cnt", ImmutableList.of("1.0", "100000001.0"), null) + : or(equality("cnt", 1.0, ColumnType.DOUBLE), equality("cnt", 1.00000001E8, ColumnType.DOUBLE)) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -4518,7 +4612,14 @@ public void testCountStarWithLongColumnFiltersOnTwoPoints() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(in("cnt", ImmutableList.of("1", "2"), null)) + .filters( + NullHandling.replaceWithDefault() + ? in("cnt", ImmutableList.of("1", "2"), null) + : or( + equality("cnt", 1L, ColumnType.LONG), + equality("cnt", 2L, ColumnType.LONG) + ) + ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -4550,13 +4651,46 @@ public void testFilterOnStringAsNumber() ) ) .setDimFilter( - or( - bound("dim1", "10", "10", false, false, null, StringComparators.NUMERIC), + NullHandling.replaceWithDefault() + ? or( + numericSelector("dim1", "10"), and( - selector("v0", "10.00", null), + selector("v0", "10.00"), bound("dim1", "9", "10.5", true, false, null, StringComparators.NUMERIC) ) ) + : or( + equality("dim1", 10L, ColumnType.LONG), + and( + equality("v0", 10.0, ColumnType.DOUBLE), + range("dim1", ColumnType.DOUBLE, 9.0, 10.5, true, false) + ) + ) + ) + .setContext(QUERY_CONTEXT_DEFAULT) + .build() + ), + ImmutableList.of( + new Object[]{"10.1"} + ) + ); + } + + @Test + public void testFilterOnStringAsNumber2() + { + testQuery( + "SELECT distinct dim1 FROM druid.foo WHERE CAST(dim1 AS float) > 9 and CAST(dim1 AS float) <= 10.5", + ImmutableList.of( + GroupByQuery.builder() + .setDataSource(CalciteTests.DATASOURCE1) + .setInterval(querySegmentSpec(Filtration.eternity())) + .setGranularity(Granularities.ALL) + .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) + .setDimFilter( + NullHandling.replaceWithDefault() + ? bound("dim1", "9", "10.5", true, false, null, StringComparators.NUMERIC) + : range("dim1", ColumnType.DOUBLE, 9.0, 10.5, true, false) ) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -4654,7 +4788,7 @@ public void testSimpleAggregations() new CountAggregatorFactory("a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("a1"), - not(selector("dim1", null, null)) + notNull("dim1") ), new LongSumAggregatorFactory("a2:sum", "cnt"), new CountAggregatorFactory("a2:count"), @@ -4663,7 +4797,7 @@ public void testSimpleAggregations() new LongMaxAggregatorFactory("a5", "cnt"), new FilteredAggregatorFactory( new CountAggregatorFactory("a6"), - not(selector("dim2", null, null)) + notNull("dim2") ), new DoubleSumAggregatorFactory("a7:sum", "d1"), new CountAggregatorFactory("a7:count") @@ -4672,32 +4806,32 @@ public void testSimpleAggregations() new CountAggregatorFactory("a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("a1"), - not(selector("cnt", null, null)) + notNull("cnt") ), new FilteredAggregatorFactory( new CountAggregatorFactory("a2"), - not(selector("dim1", null, null)) + notNull("dim1") ), new LongSumAggregatorFactory("a3:sum", "cnt"), new FilteredAggregatorFactory( new CountAggregatorFactory("a3:count"), - not(selector("cnt", null, null)) + notNull("cnt") ), new LongSumAggregatorFactory("a4", "cnt"), new LongMinAggregatorFactory("a5", "cnt"), new LongMaxAggregatorFactory("a6", "cnt"), new FilteredAggregatorFactory( new CountAggregatorFactory("a7"), - not(selector("dim2", null, null)) + notNull("dim2") ), new FilteredAggregatorFactory( new CountAggregatorFactory("a8"), - not(selector("d1", null, null)) + notNull("d1") ), new DoubleSumAggregatorFactory("a9:sum", "d1"), new FilteredAggregatorFactory( new CountAggregatorFactory("a9:count"), - not(selector("d1", null, null)) + notNull("d1") ) ) ) @@ -4885,49 +5019,49 @@ public void testFilteredAggregations() .aggregators(aggregators( new FilteredAggregatorFactory( new LongSumAggregatorFactory("a0", "cnt"), - selector("dim1", "abc", null) + equality("dim1", "abc", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a1", "cnt"), - not(selector("dim1", "abc", null)) + not(equality("dim1", "abc", ColumnType.STRING)) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a2", "cnt"), - selector("dim1", "a", new SubstringDimExtractionFn(0, 1)) + equality("dim1", "a", new SubstringDimExtractionFn(0, 1), ColumnType.STRING) ), new FilteredAggregatorFactory( new CountAggregatorFactory("a3"), and( - not(selector("dim2", null, null)), - not(selector("dim1", "1", null)) + notNull("dim2"), + not(equality("dim1", "1", ColumnType.STRING)) ) ), new FilteredAggregatorFactory( new CountAggregatorFactory("a4"), - not(selector("dim1", "1", null)) + not(equality("dim1", "1", ColumnType.STRING)) ), new FilteredAggregatorFactory( new CountAggregatorFactory("a5"), - not(selector("dim1", "1", null)) + not(equality("dim1", "1", ColumnType.STRING)) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a6", "cnt"), - selector("dim2", "a", null) + equality("dim2", "a", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a7", "cnt"), and( - selector("dim2", "a", null), - not(selector("dim1", "1", null)) + equality("dim2", "a", ColumnType.STRING), + not(equality("dim1", "1", ColumnType.STRING)) ) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a8", "cnt"), - not(selector("dim1", "1", null)) + not(equality("dim1", "1", ColumnType.STRING)) ), new FilteredAggregatorFactory( new LongMaxAggregatorFactory("a9", "cnt"), - not(selector("dim1", "1", null)) + not(equality("dim1", "1", ColumnType.STRING)) ), new FilteredAggregatorFactory( new CardinalityAggregatorFactory( @@ -4937,11 +5071,14 @@ public void testFilteredAggregations() false, true ), - not(selector("dim1", "1", null)) + not(equality("dim1", "1", ColumnType.STRING)) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a11", "cnt"), - and(selector("dim2", "a", null), selector("dim1", "b", null)) + and( + equality("dim2", "a", ColumnType.STRING), + equality("dim1", "b", ColumnType.STRING) + ) ) )) .context(QUERY_CONTEXT_DEFAULT) @@ -4975,7 +5112,7 @@ public void testCaseFilteredAggregationWithGroupBy() .setAggregatorSpecs(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("dim1", "1", null)) + not(equality("dim1", "1", ColumnType.STRING)) ), new LongSumAggregatorFactory("a1", "cnt") )) @@ -5006,13 +5143,13 @@ public void testFilteredAggregationWithNotIn() aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("dim1", "1", null)) + not(equality("dim1", "1", ColumnType.STRING)) ), new FilteredAggregatorFactory( new CountAggregatorFactory("a1"), and( - not(selector("dim2", null, null)), - not(selector("dim1", "1", null)) + notNull("dim2"), + not(equality("dim1", "1", ColumnType.STRING)) ) ) ) @@ -5097,7 +5234,7 @@ public void testExpressionFilteringAndGrouping() .setVirtualColumns( expressionVirtualColumn("v0", "(floor((\"m1\" / 2)) * 2)", ColumnType.FLOAT) ) - .setDimFilter(bound("v0", "-1", null, true, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("v0", ColumnType.LONG, -1L, null, true, false)) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.FLOAT))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setLimitSpec( @@ -5144,7 +5281,7 @@ public void testExpressionFilteringAndGroupingUsingCastToLong() expressionVirtualColumn("v0", "((CAST(\"m1\", 'LONG') / 2) * 2)", ColumnType.LONG) ) .setDimFilter( - bound("v0", "-1", null, true, false, null, StringComparators.NUMERIC) + range("v0", ColumnType.LONG, -1L, null, true, false) ) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) @@ -5196,7 +5333,7 @@ public void testExpressionFilteringAndGroupingOnStringCastToNumber() ) ) .setDimFilter( - bound("v0", "-1", null, true, false, null, StringComparators.NUMERIC) + range("v0", ColumnType.LONG, -1L, null, true, false) ) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.FLOAT))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) @@ -5268,7 +5405,14 @@ public void testSqlIsNullToInFilter() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) - .setDimFilter(new InDimFilter("dim1", Arrays.asList("abc", "def", "ghi", null), null)) + .setDimFilter( + NullHandling.replaceWithDefault() + ? in("dim1", Arrays.asList("abc", "def", "ghi", ""), null) + : or( + isNull("dim1"), + in("dim1", Arrays.asList("abc", "def", "ghi"), null) + ) + ) .setAggregatorSpecs( aggregators( new CountAggregatorFactory("a0") @@ -5337,7 +5481,7 @@ public void testCountStarWithDegenerateFilter() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - selector("dim2", "a", null) + equality("dim2", "a", ColumnType.STRING) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -5417,7 +5561,7 @@ public void testCountStarWithBoundFilterSimplifyOnMetric() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(bound("m1", "2.5", "3.5", true, true, null, StringComparators.NUMERIC)) + .filters(range("m1", ColumnType.DOUBLE, 2.5, 3.5, true, true)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -5438,7 +5582,7 @@ public void testCountStarWithBoundFilterSimplifyOr() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(bound("dim1", "a", "b", false, true, null, StringComparators.LEXICOGRAPHIC)) + .filters(range("dim1", ColumnType.STRING, "a", "b", false, true)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -5543,7 +5687,7 @@ public void testCountStarWithBoundFilterSimplifyAnd() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(selector("dim1", "abc", null)) + .filters(equality("dim1", "abc", ColumnType.STRING)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -5564,7 +5708,7 @@ public void testCountStarWithFilterOnCastedString() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(numericSelector("dim1", "2", null)) + .filters(numericEquality("dim1", 2L, ColumnType.LONG)) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -5891,12 +6035,12 @@ public void testCountStarWithComplexDisjointTimeFilter() .granularity(Granularities.ALL) .filters( and( - selector("dim2", "a", null), + equality("dim2", "a", ColumnType.STRING), or( - timeBound("2000/2001"), + timeRange("2000/2001"), and( - selector("dim1", "abc", null), - timeBound("2002-05-01/2003-05-01") + equality("dim1", "abc", ColumnType.STRING), + timeRange("2002-05-01/2003-05-01") ) ) ) @@ -5931,12 +6075,12 @@ public void testCountStarWithNotOfComplexDisjointTimeFilter() .intervals(querySegmentSpec(Filtration.eternity())) .filters( or( - not(selector("dim2", "a", null)), + not(equality("dim2", "a", ColumnType.STRING)), and( - not(timeBound("2000/2001")), + not(timeRange("2000/2001")), not(and( - selector("dim1", "abc", null), - timeBound("2002-05-01/2003-05-01") + equality("dim1", "abc", ColumnType.STRING), + timeRange("2002-05-01/2003-05-01") )) ) ) @@ -5970,7 +6114,7 @@ public void testCountStarWithNotTimeFilter() new Interval(DateTimes.of("2004"), DateTimes.MAX) ) ) - .filters(not(selector("dim1", "xxx", null))) + .filters(not(equality("dim1", "xxx", ColumnType.STRING))) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -5993,7 +6137,7 @@ public void testCountStarWithTimeAndDimFilter() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Intervals.of("2000-01-01/2001-01-01"))) - .filters(not(selector("dim2", "a", null))) + .filters(not(equality("dim2", "a", ColumnType.STRING))) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(QUERY_CONTEXT_DEFAULT) @@ -6018,15 +6162,14 @@ public void testCountStarWithTimeOrDimFilter() .intervals(querySegmentSpec(Filtration.eternity())) .filters( or( - not(selector("dim2", "a", null)), - bound( + not(equality("dim2", "a", ColumnType.STRING)), + range( "__time", - String.valueOf(timestamp("2000-01-01")), - String.valueOf(timestamp("2000-12-31T23:59:59.999")), + ColumnType.LONG, + timestamp("2000-01-01"), + timestamp("2000-12-31T23:59:59.999"), false, - false, - null, - StringComparators.NUMERIC + false ) ) ) @@ -6054,14 +6197,13 @@ public void testCountStarWithTimeFilterOnLongColumnUsingExtractEpoch() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - bound( + range( "cnt", - String.valueOf(DateTimes.of("1970-01-01").getMillis()), - String.valueOf(DateTimes.of("1970-01-02").getMillis()), + ColumnType.LONG, + DateTimes.of("1970-01-01").getMillis(), + DateTimes.of("1970-01-02").getMillis(), false, - true, - null, - StringComparators.NUMERIC + true ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -6087,14 +6229,13 @@ public void testCountStarWithTimeFilterOnLongColumnUsingExtractEpochFromDate() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - bound( + range( "cnt", - String.valueOf(DateTimes.of("1970-01-01").getMillis()), - String.valueOf(DateTimes.of("1970-01-02").getMillis()), + ColumnType.LONG, + DateTimes.of("1970-01-01").getMillis(), + DateTimes.of("1970-01-02").getMillis(), false, - true, - null, - StringComparators.NUMERIC + true ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -6120,14 +6261,13 @@ public void testCountStarWithTimeFilterOnLongColumnUsingTimestampToMillis() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .filters( - bound( + range( "cnt", - String.valueOf(DateTimes.of("1970-01-01").getMillis()), - String.valueOf(DateTimes.of("1970-01-02").getMillis()), + ColumnType.LONG, + DateTimes.of("1970-01-01").getMillis(), + DateTimes.of("1970-01-02").getMillis(), false, - true, - null, - StringComparators.NUMERIC + true ) ) .aggregators(aggregators(new CountAggregatorFactory("a0"))) @@ -6226,14 +6366,13 @@ public void testTimeseriesWithTimeFilterOnLongColumnUsingMillisToTimestamp() expressionVirtualColumn("v0", "timestamp_floor(\"cnt\",'P1Y',null,'UTC')", ColumnType.LONG) ) .setDimFilter( - bound( + range( "cnt", - String.valueOf(DateTimes.of("1970-01-01").getMillis()), - String.valueOf(DateTimes.of("1970-01-02").getMillis()), + ColumnType.LONG, + DateTimes.of("1970-01-01").getMillis(), + DateTimes.of("1970-01-02").getMillis(), false, - true, - null, - StringComparators.NUMERIC + true ) ) .setDimensions(dimensions(new DefaultDimensionSpec("v0", "d0", ColumnType.LONG))) @@ -6304,7 +6443,7 @@ public void testCountDistinctOfCaseWhen() false, true ), - bound("m1", "4", null, false, false, null, StringComparators.NUMERIC) + range("m1", ColumnType.LONG, 4L, null, false, false) ), new FilteredAggregatorFactory( new CardinalityAggregatorFactory( @@ -6314,11 +6453,11 @@ public void testCountDistinctOfCaseWhen() false, true ), - bound("m1", "4", null, false, false, null, StringComparators.NUMERIC) + range("m1", ColumnType.LONG, 4L, null, false, false) ), new FilteredAggregatorFactory( new HyperUniquesAggregatorFactory("a2", "unique_dim1", false, true), - bound("m1", "4", null, false, false, null, StringComparators.NUMERIC) + range("m1", ColumnType.LONG, 4L, null, false, false) ) ) ) @@ -6358,7 +6497,7 @@ public void testExactCountDistinct() .setAggregatorSpecs(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("d0", null, null)) + notNull("d0") ) )) .setContext(QUERY_CONTEXT_DEFAULT) @@ -6476,7 +6615,7 @@ public void testExactCountDistinctWithGroupingAndOtherAggregators() new LongSumAggregatorFactory("_a0", "a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("_a1"), - not(selector("d0", null, null)) + notNull("d0") ) )) .setContext(QUERY_CONTEXT_DEFAULT) @@ -6556,11 +6695,17 @@ public void testMultipleExactCountDistinctWithGroupingAndOtherAggregators() .setAggregatorSpecs(aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("_a0"), - and(not(selector("d1", null, null)), selector("a0", "1", null)) + and( + notNull("d1"), + equality("a0", 1L, ColumnType.LONG) + ) ), new FilteredAggregatorFactory( new CountAggregatorFactory("_a1"), - and(not(selector("d2", null, null)), selector("a0", "2", null)) + and( + notNull("d2"), + equality("a0", 2L, ColumnType.LONG) + ) ) )) .setContext(QUERY_CONTEXT_DEFAULT) @@ -6616,7 +6761,7 @@ public void testApproxCountDistinct() false, true ), - not(selector("dim2", "", null)) + not(equality("dim2", "", ColumnType.STRING)) ), new CardinalityAggregatorFactory( "a3", @@ -6718,7 +6863,7 @@ public void testNestedGroupBy() new DefaultDimensionSpec("dim1", "d1") ) ) - .setDimFilter(new SelectorDimFilter("m1", "5.0", null)) + .setDimFilter(equality("m1", 5.0, ColumnType.FLOAT)) .setAggregatorSpecs(aggregators(new LongMaxAggregatorFactory("a0", "__time"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -6742,7 +6887,7 @@ public void testNestedGroupBy() ? new CountAggregatorFactory("_a0") : new FilteredAggregatorFactory( new CountAggregatorFactory("_a0"), - not(selector("d0", null, null)) + notNull("d0") ) ) ) @@ -7007,7 +7152,7 @@ public void testAvgDailyCountDistinct() new LongSumAggregatorFactory("_a0:sum", "a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("_a0:count"), - not(selector("a0", null, null)) + notNull("a0") ) ) ) @@ -7053,6 +7198,10 @@ public void testHighestMaxNumericInFilter() @Test public void testQueryWithMoreThanMaxNumericInFilter() { + if (NullHandling.sqlCompatible()) { + // skip in sql compatible mode, this plans to an OR filter with equality filter children... + return; + } notMsqCompatible(); expectedException.expect(UOE.class); expectedException.expectMessage( @@ -7088,7 +7237,11 @@ public void testExactCountDistinctUsingSubqueryWithWherePushDown() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(not(selector("dim2", "", null))) + .setDimFilter( + not( + equality("dim2", "", ColumnType.STRING) + ) + ) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -7127,7 +7280,7 @@ public void testExactCountDistinctUsingSubqueryWithWherePushDown() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(not(selector("dim2", null, null))) + .setDimFilter(notNull("dim2")) .setGranularity(Granularities.ALL) .setDimensions(dimensions(new DefaultDimensionSpec("dim2", "d0"))) .setAggregatorSpecs(aggregators(new LongSumAggregatorFactory("a0", "cnt"))) @@ -7171,7 +7324,9 @@ public void testCompareExactAndApproximateCountDistinctUsingSubquery() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) + .setDimFilter( + not(equality("dim1", "", ColumnType.STRING)) + ) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) .setContext(QUERY_CONTEXT_DEFAULT) .build() @@ -7363,7 +7518,7 @@ public void testCountDistinctOfSubstring() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(not(selector("dim1", "", null))) + .filters(not(equality("dim1", "", ColumnType.STRING))) .granularity(Granularities.ALL) .aggregators( aggregators( @@ -7406,7 +7561,7 @@ public void testCountDistinctOfTrim() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .virtualColumns(expressionVirtualColumn("v0", "trim(\"dim1\",' ')", ColumnType.STRING)) - .filters(not(selector("v0", NullHandling.emptyToNullIfNeeded(""), null))) + .filters(not(equality("v0", "", ColumnType.STRING))) .aggregators( aggregators( new CardinalityAggregatorFactory( @@ -7478,10 +7633,9 @@ public void testRegexpExtract() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimFilter( - not(selector( + not(equality( "dim1", - "x", - new RegexDimExtractionFn("^(.)", 1, true, null) + "x", new RegexDimExtractionFn("^(.)", 1, true, null), ColumnType.STRING )) ) .setDimensions( @@ -7552,8 +7706,8 @@ public void testRegexpExtractFilterViaNotNullCheck() ) .filters( or( - not(selector("dim1", null, new RegexDimExtractionFn("^1", 0, true, null))), - not(selector("v0", null, null)) + not(isNull("dim1", new RegexDimExtractionFn("^1", 0, true, null))), + notNull("v0") ) ) .aggregators(new CountAggregatorFactory("a0")) @@ -7672,7 +7826,7 @@ public void testGroupByLimitPushDownWithHavingOnLong() 4 ) ) - .setHavingSpec(having(selector("a0", "1", null))) + .setHavingSpec(having(equality("a0", 1L, ColumnType.LONG))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -7711,7 +7865,7 @@ public void testGroupByLimitPushdownExtraction() ) ) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) - .setDimFilter(selector("dim4", "a", null)) + .setDimFilter(equality("dim4", "a", ColumnType.STRING)) .setAggregatorSpecs( aggregators( new CountAggregatorFactory("a0") @@ -7972,8 +8126,8 @@ public void testFilterOnTimeExtract() .aggregators(aggregators(new CountAggregatorFactory("a0"))) .filters( and( - selector("v0", "2000", null), - selector("v1", "1", null) + equality("v0", 2000L, ColumnType.LONG), + equality("v1", 1L, ColumnType.LONG) ) ) .context(QUERY_CONTEXT_DEFAULT) @@ -8015,8 +8169,14 @@ public void testFilterOnTimeExtractWithMultipleDays() .aggregators(aggregators(new CountAggregatorFactory("a0"))) .filters( and( - selector("v0", "2000", null), - in("v1", ImmutableList.of("2", "3", "5"), null) + equality("v0", 2000L, ColumnType.LONG), + NullHandling.replaceWithDefault() + ? in("v1", ImmutableList.of("2", "3", "5"), null) + : or( + equality("v1", 2L, ColumnType.LONG), + equality("v1", 3L, ColumnType.LONG), + equality("v1", 5L, ColumnType.LONG) + ) ) ) .context(QUERY_CONTEXT_DEFAULT) @@ -8072,14 +8232,14 @@ public void testFilterOnTimeExtractWithVariousTimeUnits() .aggregators(aggregators(new CountAggregatorFactory("a0"))) .filters( and( - selector("v0", "2000", null), - selector("v1", "946723", null), - selector("v2", "695", null), - selector("v3", "6", null), - selector("v4", "2000", null), - selector("v5", "200", null), - selector("v6", "20", null), - selector("v7", "2", null) + equality("v0", 2000L, ColumnType.LONG), + equality("v1", 946723L, ColumnType.LONG), + equality("v2", 695L, ColumnType.LONG), + equality("v3", 6L, ColumnType.LONG), + equality("v4", 2000L, ColumnType.LONG), + equality("v5", 200L, ColumnType.LONG), + equality("v6", 20L, ColumnType.LONG), + equality("v7", 2L, ColumnType.LONG) ) ) .context(QUERY_CONTEXT_DEFAULT) @@ -8215,11 +8375,14 @@ public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename() aggregators( new FilteredAggregatorFactory( new LongMinAggregatorFactory("_a0", "a0"), - selector("a1", "1", null) + equality("a1", 1L, ColumnType.LONG) ), new FilteredAggregatorFactory( new CountAggregatorFactory("_a1"), - and(not(selector("d0", null, null)), selector("a1", "0", null)) + and( + notNull("d0"), + equality("a1", 0L, ColumnType.LONG) + ) ) ) ) @@ -8472,10 +8635,9 @@ public void testFilterAndGroupByLookup() .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setDimFilter( - not(selector( + not(equality( "dim1", - "xxx", - extractionFn + "xxx", extractionFn, ColumnType.STRING )) ) .setDimensions( @@ -8625,38 +8787,35 @@ public void testFilteredTimeAggregators() .aggregators(aggregators( new FilteredAggregatorFactory( new LongSumAggregatorFactory("a0", "cnt"), - bound( + range( "__time", + ColumnType.LONG, null, - String.valueOf(timestamp("2000-02-01")), + timestamp("2000-02-01"), false, - true, - null, - StringComparators.NUMERIC + true ) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a1", "cnt"), - bound( + range( "__time", - String.valueOf(timestamp("2000-01-01T00:00:01")), - String.valueOf(timestamp("2000-02-01")), + ColumnType.LONG, + timestamp("2000-01-01T00:00:01"), + timestamp("2000-02-01"), false, - true, - null, - StringComparators.NUMERIC + true ) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a2", "cnt"), - bound( + range( "__time", - String.valueOf(timestamp("2001-01-01")), - String.valueOf(timestamp("2001-02-01")), + ColumnType.LONG, + timestamp("2001-01-01"), + timestamp("2001-02-01"), false, - true, - null, - StringComparators.NUMERIC + true ) ) )) @@ -9093,7 +9252,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValues() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .granularity(Granularities.ALL) .aggregators( aggregators( @@ -9124,14 +9283,14 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValues() ? new CountAggregatorFactory("a9:count") : new FilteredAggregatorFactory( new CountAggregatorFactory("a9:count"), - not(selector("l1", null, null)) + notNull("l1") ), new DoubleSumAggregatorFactory("a10:sum", "d1"), useDefault ? new CountAggregatorFactory("a10:count") : new FilteredAggregatorFactory( new CountAggregatorFactory("a10:count"), - not(selector("d1", null, null)) + notNull("d1") ) ) ) @@ -9200,7 +9359,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .granularity(Granularities.ALL) .aggregators( aggregators( @@ -9243,7 +9402,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("dim3", null, null)) + notNull("dim3") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -9262,7 +9421,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -9281,7 +9440,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -9300,7 +9459,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ) ) ) @@ -9337,7 +9496,7 @@ public void testGroupByAggregatorDefaultValues() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) @@ -9345,7 +9504,7 @@ public void testGroupByAggregatorDefaultValues() aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new CardinalityAggregatorFactory( @@ -9355,7 +9514,7 @@ public void testGroupByAggregatorDefaultValues() false, true ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new CardinalityAggregatorFactory( @@ -9365,57 +9524,63 @@ public void testGroupByAggregatorDefaultValues() false, true ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoubleSumAggregatorFactory("a3", "d1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoubleMaxAggregatorFactory("a4", "d1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new DoubleMinAggregatorFactory("a5", "d1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a6", "l1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongMaxAggregatorFactory("a7", "l1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongMinAggregatorFactory("a8", "l1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongSumAggregatorFactory("a9:sum", "l1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), useDefault ? new FilteredAggregatorFactory( new CountAggregatorFactory("a9:count"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) : new FilteredAggregatorFactory( new CountAggregatorFactory("a9:count"), - and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null)) + and( + notNull("l1"), + equality("dim1", "nonexistent", ColumnType.STRING) + ) ), new FilteredAggregatorFactory( new DoubleSumAggregatorFactory("a10:sum", "d1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), useDefault ? new FilteredAggregatorFactory( new CountAggregatorFactory("a10:count"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ) : new FilteredAggregatorFactory( new CountAggregatorFactory("a10:count"), - and(not(selector("d1", null, null)), selector("dim1", "nonexistent", null)) + and( + notNull("d1"), + equality("dim1", "nonexistent", ColumnType.STRING) + ) ) ) ) @@ -9487,7 +9652,7 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() GroupByQuery.builder() .setDataSource(CalciteTests.DATASOURCE3) .setInterval(querySegmentSpec(Filtration.eternity())) - .setDimFilter(selector("dim2", "a", null)) + .setDimFilter(equality("dim2", "a", ColumnType.STRING)) .setGranularity(Granularities.ALL) .setVirtualColumns(expressionVirtualColumn("v0", "'a'", ColumnType.STRING)) .setDimensions(new DefaultDimensionSpec("v0", "_d0", ColumnType.STRING)) @@ -9495,27 +9660,27 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() aggregators( new FilteredAggregatorFactory( new StringAnyAggregatorFactory("a0", "dim1", 1024), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongAnyAggregatorFactory("a1", "l1"), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new StringFirstAggregatorFactory("a2", "dim1", null, 1024), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongFirstAggregatorFactory("a3", "l1", null), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new StringLastAggregatorFactory("a4", "dim1", null, 1024), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new LongLastAggregatorFactory("a5", "l1", null), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -9534,7 +9699,7 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - selector("dim1", "nonexistent", null) + equality("dim1", "nonexistent", ColumnType.STRING) ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -9554,8 +9719,8 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() TestExprMacroTable.INSTANCE ), and( - not(selector("dim3", null, null)), - selector("dim1", "nonexistent", null) + notNull("dim3"), + equality("dim1", "nonexistent", ColumnType.STRING) ) ), new FilteredAggregatorFactory( @@ -9575,7 +9740,10 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null)) + and( + notNull("l1"), + equality("dim1", "nonexistent", ColumnType.STRING) + ) ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -9594,7 +9762,10 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null)) + and( + notNull("l1"), + equality("dim1", "nonexistent", ColumnType.STRING) + ) ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -9613,7 +9784,10 @@ public void testGroupByAggregatorDefaultValuesNonVectorized() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - and(not(selector("l1", null, null)), selector("dim1", "nonexistent", null)) + and( + notNull("l1"), + equality("dim1", "nonexistent", ColumnType.STRING) + ) ) ) ) @@ -9990,14 +10164,13 @@ public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() ) .setHavingSpec( having( - bound( + range( "a0", - "1", + ColumnType.LONG, + 1L, null, true, - false, - null, - StringComparators.NUMERIC + false ) ) ) @@ -10832,7 +11005,7 @@ public void testUnicodeFilterAndGroupBy() .setGranularity(Granularities.ALL) .setDimFilter(or( new LikeDimFilter("dim1", "דר%", null, null), - new SelectorDimFilter("dim1", "друид", null) + equality("dim1", "друид", ColumnType.STRING) )) .setDimensions(dimensions( new DefaultDimensionSpec("dim1", "d0"), @@ -11099,7 +11272,7 @@ public void testSortProjectAfterNestedGroupBy() ? new CountAggregatorFactory("a0") : new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("d1", null, null)) + notNull("d1") ) ) ) @@ -11138,7 +11311,7 @@ public void testPostAggWithTimeseries() Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .granularity(Granularities.YEAR) .aggregators( aggregators( @@ -11180,7 +11353,7 @@ public void testPostAggWithTopN() .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) .dimension(new DefaultDimensionSpec("m1", "d0", ColumnType.FLOAT)) - .filters("dim2", "a") + .filters(equality("dim2", "a", ColumnType.STRING)) .aggregators( useDefault ? aggregators( @@ -11193,7 +11366,7 @@ public void testPostAggWithTopN() new DoubleSumAggregatorFactory("a0:sum", "m2"), new FilteredAggregatorFactory( new CountAggregatorFactory("a0:count"), - not(selector("m2", null, null)) + notNull("m2") ), new DoubleSumAggregatorFactory("a1", "m1"), new DoubleSumAggregatorFactory("a2", "m2") @@ -11499,9 +11672,7 @@ public void testRequireTimeConditionPositive() ) ) ) - .setDimFilter( - not(selector("dim1", NullHandling.sqlCompatible() ? "" : null, null)) - ) + .setDimFilter(not(equality("dim1", "", ColumnType.STRING))) .setGranularity(Granularities.ALL) .setDimensions( new ExtractionDimensionSpec( @@ -11634,7 +11805,7 @@ public void testFilterFloatDimension() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1") - .filters(selector("f1", "0.1", null)) + .filters(equality("f1", 0.1, ColumnType.DOUBLE)) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) @@ -11656,7 +11827,7 @@ public void testFilterDoubleDimension() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1") - .filters(selector("d1", "1.7", null)) + .filters(equality("d1", 1.7, ColumnType.DOUBLE)) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) @@ -11678,7 +11849,7 @@ public void testFilterLongDimension() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) .columns("dim1") - .filters(selector("l1", "7", null)) + .filters(equality("l1", 7L, ColumnType.LONG)) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) @@ -11704,7 +11875,7 @@ public void testTrigonometricFunction() ImmutableList.of(Druids.newTimeseriesQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) .granularity(Granularities.ALL) .aggregators(aggregators( new CountAggregatorFactory("a0") @@ -11753,7 +11924,7 @@ public void testRadiansAndDegrees() expressionVirtualColumn("v0", "(toRadians((\"m1\" * 15)) / toDegrees(\"m2\"))", ColumnType.DOUBLE) ) .columns("v0") - .filters(selector("dim1", "1", null)) + .filters(equality("dim1", "1", ColumnType.STRING)) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -12468,7 +12639,7 @@ public void testLookupWithNull() ) .columns("v0") .legacy(false) - .filters(new SelectorDimFilter("dim2", NULL_STRING, null)) + .filters(isNull("dim2")) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) .build() @@ -12524,7 +12695,7 @@ public void testCountAndAverageByConstantVirtualColumn() aggs = ImmutableList.of( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("v0", null, null)) + notNull("v0") ), new LongSumAggregatorFactory("a1:sum", "v1", null, TestExprMacroTable.INSTANCE), new CountAggregatorFactory("a1:count") @@ -12537,12 +12708,12 @@ public void testCountAndAverageByConstantVirtualColumn() aggs = ImmutableList.of( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("v0", null, null)) + notNull("v0") ), new LongSumAggregatorFactory("a1:sum", "v1"), new FilteredAggregatorFactory( new CountAggregatorFactory("a1:count"), - not(selector("v1", null, null)) + notNull("v1") ) ); virtualColumns = ImmutableList.of( @@ -12559,8 +12730,8 @@ public void testCountAndAverageByConstantVirtualColumn() .setInterval(querySegmentSpec(Filtration.eternity())) .setDimFilter( and( - selector("dim1", "10.1", null), - selector("l1", "325323", null) + equality("dim1", "10.1", ColumnType.STRING), + equality("l1", 325323L, ColumnType.LONG) ) ) .setGranularity(Granularities.ALL) @@ -12612,15 +12783,15 @@ public void testExpressionCounts() aggregators( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("v0", null, null)) + notNull("v0") ), new FilteredAggregatorFactory( new CountAggregatorFactory("a1"), - not(selector("v1", null, null)) + notNull("v1") ), new FilteredAggregatorFactory( new CountAggregatorFactory("a2"), - not(selector("v2", null, null)) + notNull("v2") ) ) ) @@ -12671,7 +12842,7 @@ public void testBitwiseAggregatorsTimeseries() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -12690,7 +12861,7 @@ public void testBitwiseAggregatorsTimeseries() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -12709,7 +12880,7 @@ public void testBitwiseAggregatorsTimeseries() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ) ) ) @@ -12760,7 +12931,7 @@ public void testBitwiseAggregatorsGroupBy() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -12779,7 +12950,7 @@ public void testBitwiseAggregatorsGroupBy() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -12798,7 +12969,7 @@ public void testBitwiseAggregatorsGroupBy() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ) ) ) @@ -12842,7 +13013,7 @@ public void testStringAgg() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) .granularity(Granularities.ALL) - .filters(not(selector("dim1", null, null))) + .filters(notNull("dim1")) .aggregators( aggregators( new FilteredAggregatorFactory( @@ -12862,7 +13033,7 @@ public void testStringAgg() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("dim1", null, null)) + notNull("dim1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -12881,7 +13052,7 @@ public void testStringAgg() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("dim1", null, null)) + notNull("dim1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -12901,8 +13072,8 @@ public void testStringAgg() TestExprMacroTable.INSTANCE ), and( - not(selector("dim1", null, null)), - selector("dim1", "shazbot", null) + notNull("dim1"), + equality("dim1", "shazbot", ColumnType.STRING) ) ) ) @@ -12948,7 +13119,7 @@ public void testStringAggMultiValue() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("dim3", null, null)) + notNull("dim3") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -12967,7 +13138,7 @@ public void testStringAggMultiValue() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("dim3", null, null)) + notNull("dim3") ) ) ) @@ -13012,7 +13183,7 @@ public void testStringAggNumeric() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -13031,7 +13202,7 @@ public void testStringAggNumeric() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -13050,7 +13221,7 @@ public void testStringAggNumeric() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("d1", null, null)) + notNull("d1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -13069,7 +13240,7 @@ public void testStringAggNumeric() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("d1", null, null)) + notNull("d1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -13088,7 +13259,7 @@ public void testStringAggNumeric() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("f1", null, null)) + notNull("f1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -13107,7 +13278,7 @@ public void testStringAggNumeric() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("f1", null, null)) + notNull("f1") ) ) ) @@ -13169,7 +13340,7 @@ public void testStringAggExpression() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("v0", null, null)) + notNull("v0") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -13188,7 +13359,7 @@ public void testStringAggExpression() ExpressionLambdaAggregatorFactory.DEFAULT_MAX_SIZE_BYTES, TestExprMacroTable.INSTANCE ), - not(selector("v0", null, null)) + notNull("v0") ) ) ) @@ -13243,7 +13414,7 @@ public void testStringAggMaxBytes() new HumanReadableBytes(128), TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ), new FilteredAggregatorFactory( new ExpressionLambdaAggregatorFactory( @@ -13262,7 +13433,7 @@ public void testStringAggMaxBytes() new HumanReadableBytes(128), TestExprMacroTable.INSTANCE ), - not(selector("l1", null, null)) + notNull("l1") ) ) ) @@ -13327,7 +13498,7 @@ public void testHumanReadableFormatFunction() expressionVirtualColumn("v7", "human_readable_decimal_format(\"l1\")", ColumnType.STRING) ) .columns("m1", "v0", "v1", "v2", "v3", "v4", "v5", "v6", "v7") - .filters(selector("dim1", "1", null)) + .filters(equality("dim1", "1", ColumnType.STRING)) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(1) .context(QUERY_CONTEXT_DEFAULT) @@ -13410,7 +13581,7 @@ public void testCommonVirtualExpressionWithDifferentValueType() ImmutableList.of(new TopNQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim1", "none", null)) + .filters(equality("dim1", "none", ColumnType.STRING)) .granularity(Granularities.ALL) .virtualColumns( expressionVirtualColumn("v0", "'none'", ColumnType.STRING), @@ -13447,8 +13618,10 @@ public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithSingleConsta .intervals(querySegmentSpec(Filtration.eternity())) .filters( and( - selector("m1", "50", null), - selector("dim1", "wat", null) + NullHandling.replaceWithDefault() + ? selector("m1", "50") + : equality("m1", 50.0, ColumnType.FLOAT), + equality("dim1", "wat", ColumnType.STRING) ) ) .granularity(Granularities.ALL) @@ -13474,7 +13647,7 @@ public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithSingleConsta .setInterval(querySegmentSpec(Intervals.ETERNITY)) .setGranularity(Granularities.ALL) .addDimension(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING)) - .setDimFilter(selector("dim1", "wat", null)) + .setDimFilter(equality("dim1", "wat", ColumnType.STRING)) .setPostAggregatorSpecs( ImmutableList.of( new ExpressionPostAggregator("p0", "'A'", null, ExprMacroTable.nil()) @@ -13499,8 +13672,10 @@ public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithMultipleCons .intervals(querySegmentSpec(Filtration.eternity())) .filters( and( - selector("m1", "50", null), - selector("dim1", "wat", null) + NullHandling.replaceWithDefault() + ? selector("m1", "50") + : equality("m1", 50.0, ColumnType.FLOAT), + equality("dim1", "wat", ColumnType.STRING) ) ) .granularity(Granularities.ALL) @@ -13524,8 +13699,8 @@ public void testReturnEmptyRowWhenGroupByIsConvertedToTimeseriesWithMultipleCons .intervals(querySegmentSpec(Filtration.eternity())) .filters( and( - selector("m1", "2.0", null), - selector("dim1", "10.1", null) + equality("m1", 2.0, ColumnType.DOUBLE), + equality("dim1", "10.1", ColumnType.STRING) ) ) .granularity(Granularities.ALL) @@ -13559,10 +13734,15 @@ public void testPlanWithInFilterLessThanInSubQueryThreshold() .context(QUERY_CONTEXT_DEFAULT) .legacy(false) .filters( - in( - "l1", - ImmutableList.of("4842", "4844", "4845", "14905", "4853", "29064"), - null + NullHandling.replaceWithDefault() + ? in("l1", ImmutableList.of("4842", "4844", "4845", "14905", "4853", "29064"), null) + : or( + equality("l1", 4842L, ColumnType.LONG), + equality("l1", 4844L, ColumnType.LONG), + equality("l1", 4845L, ColumnType.LONG), + equality("l1", 14905L, ColumnType.LONG), + equality("l1", 4853L, ColumnType.LONG), + equality("l1", 29064L, ColumnType.LONG) ) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -13947,10 +14127,10 @@ public void testFilterWithNVLAndNotIn() .columns(ImmutableList.of("__time", "dim1")) .filters(and( or( - not(selector("dim1", "a", null)), - selector("dim1", null, null) + not(equality("dim1", "a", ColumnType.STRING)), + isNull("dim1") ), - not(selector("dim1", NullHandling.sqlCompatible() ? "" : null, null)) + not(equality("dim1", "", ColumnType.STRING)) )) .build() ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java index c4f9503d4c7a..2970330e82fc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteReplaceDmlTest.java @@ -328,7 +328,7 @@ public void testReplaceFromView() .dataSource("foo") .intervals(querySegmentSpec(Filtration.eternity())) .virtualColumns(expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ColumnType.STRING)) - .filters(selector("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .columns("v0") .context(REPLACE_ALL_TIME_CHUNKS) .build() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java index cc60a27acdd1..be05b46c3716 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSelectQueryTest.java @@ -592,8 +592,8 @@ public void testSelectStarWithDimFilter() .intervals(querySegmentSpec(Filtration.eternity())) .filters( or( - bound("dim1", "d", null, true, false, null, StringComparators.LEXICOGRAPHIC), - selector("dim2", "a", null) + range("dim1", ColumnType.STRING, "d", null, true, false), + equality("dim2", "a", ColumnType.STRING) ) ) .columns("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1") @@ -622,15 +622,14 @@ public void testSelectDistinctWithCascadeExtractionFilter() .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) .setDimFilter( or( - selector( + equality( "dim1", - "e", - cascade( + "e", cascade( new SubstringDimExtractionFn(1, null), new SubstringDimExtractionFn(0, 1) - ) + ), ColumnType.STRING ), - selector("dim2", "a", null) + equality("dim2", "a", ColumnType.STRING) ) ) .setContext(QUERY_CONTEXT_DEFAULT) @@ -670,8 +669,8 @@ public void testSelectDistinctWithStrlenFilter() .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0"))) .setDimFilter( or( - selector("v0", "3", null), - selector("v1", "3", null) + equality("v0", 3L, ColumnType.LONG), + equality("v1", 3L, ColumnType.LONG) ) ) .setContext(QUERY_CONTEXT_DEFAULT) @@ -987,19 +986,19 @@ public void testSelectCountStar() "SELECT exp(count(*)) + 10, sum(m2) FROM druid.foo WHERE dim2 = 0", CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of(Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .filters(bound("dim2", "0", "0", false, false, null, StringComparators.NUMERIC)) - .granularity(Granularities.ALL) - .aggregators(aggregators( - new CountAggregatorFactory("a0"), - new DoubleSumAggregatorFactory("a1", "m2") - )) - .postAggregators( - expressionPostAgg("p0", "(exp(\"a0\") + 10)") - ) - .context(QUERY_CONTEXT_DEFAULT) - .build()), + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(numericEquality("dim2", 0L, ColumnType.LONG)) + .granularity(Granularities.ALL) + .aggregators(aggregators( + new CountAggregatorFactory("a0"), + new DoubleSumAggregatorFactory("a1", "m2") + )) + .postAggregators( + expressionPostAgg("p0", "(exp(\"a0\") + 10)") + ) + .context(QUERY_CONTEXT_DEFAULT) + .build()), ImmutableList.of( new Object[]{11.0, NullHandling.defaultDoubleValue()} ) @@ -1038,15 +1037,15 @@ public void testSelectCountStar() "SELECT COUNT(*) FROM foo WHERE dim1 = 'nonexistent'", CalciteTests.REGULAR_USER_AUTH_RESULT, ImmutableList.of(Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim1", "nonexistent", null)) - .granularity(Granularities.ALL) - .aggregators(aggregators( - new CountAggregatorFactory("a0") - )) - .context(TIMESERIES_CONTEXT_BY_GRAN) - .build()), + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(equality("dim1", "nonexistent", ColumnType.STRING)) + .granularity(Granularities.ALL) + .aggregators(aggregators( + new CountAggregatorFactory("a0") + )) + .context(TIMESERIES_CONTEXT_BY_GRAN) + .build()), ImmutableList.of() ); @@ -1054,15 +1053,15 @@ public void testSelectCountStar() testQuery( "SELECT COUNT(*) FROM foo WHERE dim1 = 'nonexistent' GROUP BY FLOOR(__time TO DAY)", ImmutableList.of(Druids.newTimeseriesQueryBuilder() - .dataSource(CalciteTests.DATASOURCE1) - .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim1", "nonexistent", null)) - .granularity(Granularities.DAY) - .aggregators(aggregators( - new CountAggregatorFactory("a0") - )) - .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0")) - .build()), + .dataSource(CalciteTests.DATASOURCE1) + .intervals(querySegmentSpec(Filtration.eternity())) + .filters(equality("dim1", "nonexistent", ColumnType.STRING)) + .granularity(Granularities.DAY) + .aggregators(aggregators( + new CountAggregatorFactory("a0") + )) + .context(getTimeseriesContextWithFloorTime(TIMESERIES_CONTEXT_BY_GRAN, "d0")) + .build()), ImmutableList.of() ); } @@ -1181,7 +1180,7 @@ public void testSelectStarOnForbiddenView() expressionVirtualColumn("v0", "substring(\"dim1\", 0, 1)", ColumnType.STRING), expressionVirtualColumn("v1", "'a'", ColumnType.STRING) ) - .filters(selector("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .columns("__time", "v0", "v1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) @@ -1212,7 +1211,7 @@ public void testSelectStarOnRestrictedView() ImmutableList.of( newScanQueryBuilder() .dataSource(CalciteTests.FORBIDDEN_DATASOURCE) - .filters(selector("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .intervals(querySegmentSpec(Filtration.eternity())) .columns("__time", "dim1", "dim2", "m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -1237,7 +1236,7 @@ public void testSelectStarOnRestrictedView() newScanQueryBuilder() .dataSource(CalciteTests.FORBIDDEN_DATASOURCE) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(selector("dim2", "a", null)) + .filters(equality("dim2", "a", ColumnType.STRING)) .columns("__time", "dim1", "dim2", "m1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) @@ -1412,7 +1411,11 @@ public void testSelectWithExpressionFilter() .virtualColumns( expressionVirtualColumn("v0", "(\"m1\" + 1)", ColumnType.FLOAT) ) - .filters(selector("v0", "7", null)) + .filters( + NullHandling.replaceWithDefault() + ? selector("v0", "7") + : equality("v0", 7.0, ColumnType.FLOAT) + ) .columns("dim1") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .context(QUERY_CONTEXT_DEFAULT) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java index 40c04f358073..21853f3c3942 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -132,7 +132,7 @@ public void testExactCountDistinctUsingSubqueryWithWhereToOuterFilter() .build() ) ) - .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("a0", ColumnType.LONG, 0L, null, true, false)) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setAggregatorSpecs(aggregators( @@ -182,7 +182,7 @@ public void testExactCountDistinctOfSemiJoinResult() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(not(selector("dim1", "", null))) + .setDimFilter(not(equality("dim1", "", ColumnType.STRING))) .setDimensions( dimensions( new ExtractionDimensionSpec( @@ -258,7 +258,7 @@ public void testTwoExactCountDistincts() .setAggregatorSpecs( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("d0", null, null)) + notNull("d0") ) ) .setContext(QUERY_CONTEXT_DEFAULT) @@ -282,7 +282,7 @@ public void testTwoExactCountDistincts() .setAggregatorSpecs( new FilteredAggregatorFactory( new CountAggregatorFactory("a0"), - not(selector("d0", null, null)) + notNull("d0") ) ) .setContext(QUERY_CONTEXT_DEFAULT) @@ -329,7 +329,7 @@ public void testViewAndJoin() "j0.", "(\"dim2\" == \"j0.dim2\")", JoinType.INNER, - bound("dim2", "a", "a", false, false, null, null) + range("dim2", ColumnType.STRING, "a", "a", false, false) ), new QueryDataSource( newScanQueryBuilder().dataSource(CalciteTests.DATASOURCE1) @@ -344,7 +344,7 @@ public void testViewAndJoin() ) ) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(not(selector("dim1", "z", new SubstringDimExtractionFn(0, 1)))) + .filters(not(equality("dim1", "z", new SubstringDimExtractionFn(0, 1), ColumnType.STRING))) .granularity(Granularities.ALL) .aggregators(aggregators(new CountAggregatorFactory("a0"))) .context(queryContextModified) @@ -481,7 +481,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() .setDataSource(CalciteTests.DATASOURCE1) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(selector("dim2", "abc", null)) + .setDimFilter(equality("dim2", "abc", ColumnType.STRING)) .setDimensions(dimensions( new DefaultDimensionSpec("dim1", "d0"), new DefaultDimensionSpec("dim2", "d1") @@ -490,7 +490,7 @@ public void testUsingSubqueryAsFilterOnTwoColumns() .setPostAggregatorSpecs( ImmutableList.of(expressionPostAgg("p0", "'abc'")) ) - .setHavingSpec(having(selector("a0", "1", null))) + .setHavingSpec(having(equality("a0", 1L, ColumnType.LONG))) .setContext(QUERY_CONTEXT_DEFAULT) .build() ), @@ -561,7 +561,7 @@ public void testMinMaxAvgDailyCountWithLimit() new LongSumAggregatorFactory("_a2:sum", "a0"), new FilteredAggregatorFactory( new CountAggregatorFactory("_a2:count"), - not(selector("a0", null, null)) + notNull("a0") ), new LongMaxAggregatorFactory("_a3", "d0"), new CountAggregatorFactory("_a4") @@ -697,7 +697,7 @@ public void testMaxSubqueryRows() .build() ) ) - .setDimFilter(bound("a0", "0", null, true, false, null, StringComparators.NUMERIC)) + .setDimFilter(range("a0", ColumnType.LONG, 0L, null, true, false)) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) .setAggregatorSpecs(aggregators(