From b0a9c318d64b8f054c46c02376d20ca97cb56477 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 22 Mar 2024 12:45:08 -0700 Subject: [PATCH] add new typed in filter (#16039) changes: * adds TypedInFilter which preserves matching sets in the native match value type * SQL planner uses new TypedInFilter when druid.generic.useDefaultValueForNull=false (the default) --- ...ryEncodedStringIndexSupplierBenchmark.java | 14 +- .../druid/benchmark/InFilterBenchmark.java | 43 +- .../druid/benchmark/query/SqlBenchmark.java | 7 +- .../query/SqlNestedDataBenchmark.java | 64 +- .../druid/query/filter/BoundDimFilter.java | 3 + .../apache/druid/query/filter/DimFilter.java | 3 +- .../druid/query/filter/DimFilterUtils.java | 1 + .../druid/query/filter/InDimFilter.java | 16 +- .../druid/query/filter/SelectorDimFilter.java | 2 +- .../druid/query/filter/TypedInFilter.java | 756 ++++++++++++++ .../segment/index/BitmapColumnIndex.java | 4 +- .../index/IndexedUtf8ValueIndexes.java | 242 ++--- .../index/semantic/ArrayElementIndexes.java | 4 + .../semantic/DictionaryEncodedValueIndex.java | 1 - .../index/semantic/DruidPredicateIndexes.java | 3 +- .../index/semantic/NullValueIndex.java | 2 +- .../index/semantic/StringValueSetIndexes.java | 10 +- .../index/semantic/Utf8ValueSetIndexes.java | 11 +- .../segment/index/semantic/ValueIndexes.java | 10 +- .../index/semantic/ValueSetIndexes.java | 331 ++++++ .../ScalarDoubleColumnAndIndexSupplier.java | 74 ++ .../ScalarLongColumnAndIndexSupplier.java | 76 +- .../serde/StringUtf8ColumnIndexSupplier.java | 4 +- .../druid/segment/filter/BoundFilterTest.java | 4 + .../druid/segment/filter/InFilterTest.java | 585 ----------- .../druid/segment/filter/InFilterTests.java | 978 ++++++++++++++++++ .../segment/filter/SelectorFilterTest.java | 4 + .../ArrayOverlapOperatorConversion.java | 52 +- .../filtration/CollectComparisons.java | 12 +- .../filtration/ConvertSelectorsToIns.java | 100 +- .../sql/calcite/rule/ReverseLookupRule.java | 8 +- .../sql/calcite/BaseCalciteQueryTest.java | 23 +- .../sql/calcite/CalciteArraysQueryTest.java | 57 +- .../sql/calcite/CalciteJoinQueryTest.java | 68 +- .../CalciteLookupFunctionQueryTest.java | 58 +- .../CalciteMultiValueStringQueryTest.java | 9 +- .../calcite/CalciteNestedDataQueryTest.java | 19 +- .../calcite/CalciteParameterQueryTest.java | 7 +- .../druid/sql/calcite/CalciteQueryTest.java | 91 +- .../sql/calcite/CalciteSelectQueryTest.java | 2 +- .../sql/calcite/CalciteSubqueryTest.java | 10 +- .../sql/calcite/CalciteUnionQueryTest.java | 10 +- 42 files changed, 2776 insertions(+), 1002 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/filter/TypedInFilter.java create mode 100644 processing/src/main/java/org/apache/druid/segment/index/semantic/ValueSetIndexes.java delete mode 100644 processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java 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 c80a618a84bb..8491a1283296 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DictionaryEncodedStringIndexSupplierBenchmark.java @@ -25,11 +25,11 @@ import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.java.util.common.ByteBufferUtils; import org.apache.druid.java.util.common.StringUtils; 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; @@ -73,7 +73,7 @@ public static class BenchmarkState { @Nullable private IndexedUtf8ValueIndexes stringValueSetIndex; - private final TreeSet values = new TreeSet<>(); + private final List values = new ArrayList<>(); private static final int START_INT = 10_000_000; // cardinality of the dictionary. it will contain this many ints (as strings, of course), starting at START_INT, @@ -122,14 +122,16 @@ public void setup() Random r = new Random(9001); Collections.shuffle(filterValues); Collections.shuffle(nonFilterValues); - values.clear(); + TreeSet sortedValues = new TreeSet<>(ByteBufferUtils.utf8Comparator()); for (int i = 0; i < filterToDictionaryPercentage * dictionarySize / 100; i++) { if (r.nextInt(100) < selectivityPercentage) { - values.add(ByteBuffer.wrap((filterValues.get(i).toString()).getBytes(StandardCharsets.UTF_8))); + sortedValues.add(ByteBuffer.wrap((filterValues.get(i).toString()).getBytes(StandardCharsets.UTF_8))); } else { - values.add(ByteBuffer.wrap((nonFilterValues.get(i).toString()).getBytes(StandardCharsets.UTF_8))); + sortedValues.add(ByteBuffer.wrap((nonFilterValues.get(i).toString()).getBytes(StandardCharsets.UTF_8))); } } + values.clear(); + values.addAll(sortedValues); } private Iterable intGenerator() @@ -144,6 +146,6 @@ private Iterable intGenerator() @OutputTimeUnit(TimeUnit.MICROSECONDS) public void doValueSetCheck(Blackhole blackhole, BenchmarkState state) { - BitmapColumnIndex bitmapIndex = state.stringValueSetIndex.forSortedValuesUtf8(state.values); + blackhole.consume(state.stringValueSetIndex.forSortedValuesUtf8(state.values)); } } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java index 7bdcaf1c6abd..821209f6d8a4 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/InFilterBenchmark.java @@ -28,6 +28,8 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.TypedInFilter; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; @@ -53,8 +55,8 @@ @State(Scope.Benchmark) @Fork(value = 1) -@Warmup(iterations = 10) -@Measurement(iterations = 10) +@Warmup(iterations = 2) +@Measurement(iterations = 3) public class InFilterBenchmark { static { @@ -65,6 +67,8 @@ public class InFilterBenchmark private InDimFilter inFilter; private InDimFilter endInDimFilter; + private TypedInFilter newInFilter; + private TypedInFilter newEndInFilter; // cardinality of the dictionary. it will contain this many ints (as strings, of course), starting at START_INT, // even numbers only. @@ -110,12 +114,29 @@ public void setup() "dummy", IntStream.range(START_INT, START_INT + filterSize).mapToObj(String::valueOf).collect(Collectors.toSet()) ); + newInFilter = (TypedInFilter) new TypedInFilter( + "dummy", + ColumnType.STRING, + IntStream.range(START_INT, START_INT + filterSize).mapToObj(String::valueOf).collect(Collectors.toList()), + null, + null + ).toFilter(); endInDimFilter = new InDimFilter( "dummy", IntStream.range(START_INT + dictionarySize * 2, START_INT + dictionarySize * 2 + 1) .mapToObj(String::valueOf) .collect(Collectors.toSet()) ); + + newEndInFilter = (TypedInFilter) new TypedInFilter( + "dummy", + ColumnType.STRING, + IntStream.range(START_INT + dictionarySize * 2, START_INT + dictionarySize * 2 + 1) + .mapToObj(String::valueOf) + .collect(Collectors.toList()), + null, + null + ).toFilter(); } @Benchmark @@ -136,6 +157,24 @@ public void doFilterAtEnd(Blackhole blackhole) blackhole.consume(bitmapIndex); } + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void doFilter2(Blackhole blackhole) + { + final ImmutableBitmap bitmapIndex = Filters.computeDefaultBitmapResults(newInFilter, selector); + blackhole.consume(bitmapIndex); + } + + @Benchmark + @BenchmarkMode(Mode.AverageTime) + @OutputTimeUnit(TimeUnit.MICROSECONDS) + public void doFilterAtEnd2(Blackhole blackhole) + { + final ImmutableBitmap bitmapIndex = Filters.computeDefaultBitmapResults(newEndInFilter, selector); + blackhole.consume(bitmapIndex); + } + private Iterable intGenerator() { // i * 2 => half of these values will be present in the inFilter, half won't. diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 60300d88d0cc..f04f7438d7bf 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -583,8 +583,8 @@ public void setup() .writeValueAsString(jsonMapper.readValue((String) planResult[0], List.class)) ); } - catch (JsonProcessingException e) { - throw new RuntimeException(e); + catch (JsonProcessingException ignored) { + } try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, ImmutableMap.of())) { @@ -598,6 +598,9 @@ public void setup() } log.info("Total result row count:" + rowCounter); } + catch (Throwable ignored) { + + } } private StringEncodingStrategy getStringEncodingStrategy() diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java index 1628babb97c4..0be1f4d52e6a 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java @@ -19,6 +19,9 @@ package org.apache.druid.benchmark.query; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -28,6 +31,8 @@ import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExpressionProcessing; @@ -198,7 +203,17 @@ public String getFormatString() "SELECT SUM(long1) FROM foo WHERE string5 LIKE '%1%' AND string1 = '1000'", "SELECT SUM(JSON_VALUE(nested, '$.long1' RETURNING BIGINT)) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.string5') LIKE '%1%' AND JSON_VALUE(nested, '$.nesteder.string1') = '1000'", "SELECT SUM(long1) FROM foo WHERE string1 = '1000' AND string5 LIKE '%1%'", - "SELECT SUM(JSON_VALUE(nested, '$.long1' RETURNING BIGINT)) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.string1') = '1000' AND JSON_VALUE(nested, '$.nesteder.string5') LIKE '%1%'" + "SELECT SUM(JSON_VALUE(nested, '$.long1' RETURNING BIGINT)) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.string1') = '1000' AND JSON_VALUE(nested, '$.nesteder.string5') LIKE '%1%'", + //48,49 bigger in + "SELECT long2 FROM foo WHERE long2 IN (1, 19, 21, 23, 25, 26, 46, 50, 51, 55, 60, 61, 66, 68, 69, 70, 77, 88, 90, 92, 93, 94, 95, 100, 101, 102, 104, 109, 111, 113, 114, 115, 120, 121, 122, 134, 135, 136, 140, 142, 150, 155, 170, 172, 173, 174, 180, 181, 190, 199, 200, 201, 202, 203, 204)", + "SELECT JSON_VALUE(nested, '$.nesteder.long2' RETURNING BIGINT) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.long2' RETURNING BIGINT) IN (1, 19, 21, 23, 25, 26, 46, 50, 51, 55, 60, 61, 66, 68, 69, 70, 77, 88, 90, 92, 93, 94, 95, 100, 101, 102, 104, 109, 111, 113, 114, 115, 120, 121, 122, 134, 135, 136, 140, 142, 150, 155, 170, 172, 173, 174, 180, 181, 190, 199, 200, 201, 202, 203, 204)", + //50, 51 bigger in group + "SELECT long2 FROM foo WHERE long2 IN (1, 19, 21, 23, 25, 26, 46, 50, 51, 55, 60, 61, 66, 68, 69, 70, 77, 88, 90, 92, 93, 94, 95, 100, 101, 102, 104, 109, 111, 113, 114, 115, 120, 121, 122, 134, 135, 136, 140, 142, 150, 155, 170, 172, 173, 174, 180, 181, 190, 199, 200, 201, 202, 203, 204) GROUP BY 1", + "SELECT JSON_VALUE(nested, '$.nesteder.long2' RETURNING BIGINT) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.long2' RETURNING BIGINT) IN (1, 19, 21, 23, 25, 26, 46, 50, 51, 55, 60, 61, 66, 68, 69, 70, 77, 88, 90, 92, 93, 94, 95, 100, 101, 102, 104, 109, 111, 113, 114, 115, 120, 121, 122, 134, 135, 136, 140, 142, 150, 155, 170, 172, 173, 174, 180, 181, 190, 199, 200, 201, 202, 203, 204) GROUP BY 1", + "SELECT long2 FROM foo WHERE double3 IN (1.0, 19.0, 21.0, 23.0, 25.0, 26.0, 46.0, 50.0, 51.0, 55.0, 60.0, 61.0, 66.0, 68.0, 69.0, 70.0, 77.0, 88.0, 90.0, 92.0, 93.0, 94.0, 95.0, 100.0, 101.0, 102.0, 104.0, 109.0, 111.0, 113.0, 114.0, 115.0, 120.0, 121.0, 122.0, 134.0, 135.0, 136.0, 140.0, 142.0, 150.0, 155.0, 170.0, 172.0, 173.0, 174.0, 180.0, 181.0, 190.0, 199.0, 200.0, 201.0, 202.0, 203.0, 204.0)", + "SELECT JSON_VALUE(nested, '$.nesteder.long2' RETURNING BIGINT) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.double3' RETURNING DOUBLE) IN (1.0, 19.0, 21.0, 23.0, 25.0, 26.0, 46.0, 50.0, 51.0, 55.0, 60.0, 61.0, 66.0, 68.0, 69.0, 70.0, 77.0, 88.0, 90.0, 92.0, 93.0, 94.0, 95.0, 100.0, 101.0, 102.0, 104.0, 109.0, 111.0, 113.0, 114.0, 115.0, 120.0, 121.0, 122.0, 134.0, 135.0, 136.0, 140.0, 142.0, 150.0, 155.0, 170.0, 172.0, 173.0, 174.0, 180.0, 181.0, 190.0, 199.0, 200.0, 201.0, 202.0, 203.0, 204.0)", + "SELECT long2 FROM foo WHERE double3 IN (1.0, 19.0, 21.0, 23.0, 25.0, 26.0, 46.0, 50.0, 51.0, 55.0, 60.0, 61.0, 66.0, 68.0, 69.0, 70.0, 77.0, 88.0, 90.0, 92.0, 93.0, 94.0, 95.0, 100.0, 101.0, 102.0, 104.0, 109.0, 111.0, 113.0, 114.0, 115.0, 120.0, 121.0, 122.0, 134.0, 135.0, 136.0, 140.0, 142.0, 150.0, 155.0, 170.0, 172.0, 173.0, 174.0, 180.0, 181.0, 190.0, 199.0, 200.0, 201.0, 202.0, 203.0, 204.0) GROUP BY 1", + "SELECT JSON_VALUE(nested, '$.nesteder.long2' RETURNING BIGINT) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.double3' RETURNING DOUBLE) IN (1.0, 19.0, 21.0, 23.0, 25.0, 26.0, 46.0, 50.0, 51.0, 55.0, 60.0, 61.0, 66.0, 68.0, 69.0, 70.0, 77.0, 88.0, 90.0, 92.0, 93.0, 94.0, 95.0, 100.0, 101.0, 102.0, 104.0, 109.0, 111.0, 113.0, 114.0, 115.0, 120.0, 121.0, 122.0, 134.0, 135.0, 136.0, 140.0, 142.0, 150.0, 155.0, 170.0, 172.0, 173.0, 174.0, 180.0, 181.0, 190.0, 199.0, 200.0, 201.0, 202.0, 203.0, 204.0) GROUP BY 1" ); @Param({"5000000"}) @@ -271,7 +286,15 @@ public String getFormatString() "44", "45", "46", - "47" + "47", + "48", + "49", + "50", + "51", + "52", + "53", + "54", + "55" }) private String query; @@ -386,8 +409,41 @@ public void setup() QUERIES.get(Integer.parseInt(query)) ); } - catch (Throwable ignored) { - // the show must go on + catch (Throwable ex) { + log.warn(ex, "failed to sanity check"); + } + + final String sql = QUERIES.get(Integer.parseInt(query)); + final ObjectMapper jsonMapper = CalciteTests.getJsonMapper(); + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, "EXPLAIN PLAN FOR " + sql, ImmutableMap.of("useNativeQueryExplain", true))) { + final PlannerResult plannerResult = planner.plan(); + final Sequence resultSequence = plannerResult.run().getResults(); + final Object[] planResult = resultSequence.toList().get(0); + log.info("Native query plan:\n" + + jsonMapper.writerWithDefaultPrettyPrinter() + .writeValueAsString(jsonMapper.readValue((String) planResult[0], List.class)) + ); + } + catch (JsonMappingException e) { + throw new RuntimeException(e); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, ImmutableMap.of())) { + final PlannerResult plannerResult = planner.plan(); + final Sequence resultSequence = plannerResult.run().getResults(); + final Yielder yielder = Yielders.each(resultSequence); + int rowCounter = 0; + while (!yielder.isDone()) { + rowCounter++; + yielder.next(yielder.get()); + } + log.info("Total result row count:" + rowCounter); + } + catch (Throwable ex) { + log.warn(ex, "failed to count rows"); } } 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 c5e0bee077c4..311595e99e87 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 @@ -47,6 +47,9 @@ import java.util.Objects; import java.util.Set; +/** + * Recommended to use {@link RangeFilter} instead + */ public class BoundDimFilter extends AbstractOptimizableDimFilter implements DimFilter { private final String dimension; 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 e78bc9870290..99ad72f0ee93 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 @@ -55,7 +55,8 @@ @JsonSubTypes.Type(name = "range", value = RangeFilter.class), @JsonSubTypes.Type(name = "isfalse", value = IsFalseDimFilter.class), @JsonSubTypes.Type(name = "istrue", value = IsTrueDimFilter.class), - @JsonSubTypes.Type(name = "arrayContainsElement", value = ArrayContainsElementFilter.class) + @JsonSubTypes.Type(name = "arrayContainsElement", value = ArrayContainsElementFilter.class), + @JsonSubTypes.Type(name = "inType", value = TypedInFilter.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 b60ac9572cac..dad5cfd98b30 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 @@ -62,6 +62,7 @@ public class DimFilterUtils static final byte RANGE_CACHE_ID = 0x14; static final byte IS_FILTER_BOOLEAN_FILTER_CACHE_ID = 0x15; static final byte ARRAY_CONTAINS_CACHE_ID = 0x16; + static final byte TYPED_IN_CACHE_ID = 0x17; public static final byte STRING_SEPARATOR = (byte) 0xFF; 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 035fd18eeabb..d678f4b53f12 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 @@ -31,6 +31,7 @@ import com.google.common.collect.ForwardingSortedSet; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; import com.google.common.collect.Range; import com.google.common.collect.RangeSet; import com.google.common.collect.Sets; @@ -42,7 +43,6 @@ import it.unimi.dsi.fastutil.longs.LongArrayList; import it.unimi.dsi.fastutil.longs.LongOpenHashSet; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.java.util.common.ByteBufferUtils; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Comparators; @@ -69,12 +69,20 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; +/** + * Approximately like the SQL 'IN' filter, with the main difference being that this will match NULL values if contained + * in the values list instead of ignoring them. + *

+ * This filter specifies all match values as a sorted string set; matching against other column types must incur the + * cost of converting values to check for matches. For the most part, {@link TypedInFilter} should be used instead. + */ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter { /** @@ -84,7 +92,7 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter */ private final ValuesSet values; // Computed eagerly, not lazily, because lazy computations would block all processing threads for a given query. - private final SortedSet valuesUtf8; + private final List valuesUtf8; private final String dimension; @Nullable private final ExtractionFn extractionFn; @@ -806,9 +814,9 @@ public static ValuesSet copyOf(final Collection values) return copyOf(values.iterator()); } - public SortedSet toUtf8() + public List toUtf8() { - final TreeSet valuesUtf8 = new TreeSet<>(ByteBufferUtils.utf8Comparator()); + final List valuesUtf8 = Lists.newArrayListWithCapacity(values.size()); for (final String value : values) { if (value == null) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java index 4489a9fa601d..05cd4730e4b9 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/SelectorDimFilter.java @@ -39,7 +39,7 @@ import java.util.Set; /** - * + * Recommended to use {@link EqualityFilter} or {@link NullFilter} instead */ public class SelectorDimFilter extends AbstractOptimizableDimFilter implements DimFilter { diff --git a/processing/src/main/java/org/apache/druid/query/filter/TypedInFilter.java b/processing/src/main/java/org/apache/druid/query/filter/TypedInFilter.java new file mode 100644 index 000000000000..d3b307ef2f42 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/TypedInFilter.java @@ -0,0 +1,756 @@ +/* + * 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.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Range; +import com.google.common.collect.RangeSet; +import com.google.common.collect.Sets; +import com.google.common.collect.TreeRangeSet; +import com.google.common.hash.Hasher; +import com.google.common.hash.Hashing; +import it.unimi.dsi.fastutil.doubles.DoubleOpenHashSet; +import it.unimi.dsi.fastutil.floats.FloatOpenHashSet; +import it.unimi.dsi.fastutil.longs.LongOpenHashSet; +import it.unimi.dsi.fastutil.objects.ObjectArrays; +import org.apache.druid.common.config.NullHandling; +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.Evals; +import org.apache.druid.query.cache.CacheKeyBuilder; +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.ColumnSelectorFactory; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.column.ColumnIndexSupplier; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.NullableTypeStrategy; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.semantic.Utf8ValueSetIndexes; +import org.apache.druid.segment.index.semantic.ValueSetIndexes; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * Approximately like the SQL 'IN' filter, with the main difference being that this will match NULL values if contained + * in the values list instead of ignoring them. + *

+ * This is a typed version of {@link InDimFilter}, which allows the match values to exist in their native type and + * sorted in their type native order for better performance matching against all column types. + */ +public class TypedInFilter extends AbstractOptimizableDimFilter implements Filter +{ + /** + * Column to match {@link #sortedMatchValues} or {@link #sortedUtf8MatchValueBytes} against. + */ + private final String column; + + /** + * Type of values contained in {@link #sortedMatchValues}. This might be the same or different than the + * {@link ColumnType} of {@link #column}, but is encouraged to be the same there are several optimizations available + * if they match. + */ + private final ColumnType matchValueType; + + /** + * Unsorted values. This will be null if the values are found to be sorted, or have been already sorted "upstream". + * Otherwise, this set of values will be lazily computed into {@link #sortedMatchValues} as needed, e.g. for + * JSON serialization, cache key building, building a hashcode, or checking equality. + */ + @Nullable + private final List unsortedValues; + + /** + * Supplier for list of values sorted by {@link #matchValueType}. This is lazily computed if + * {@link #unsortedValues} is not null and previously sorted. Data will be deduplicated upon sorting if computed. + * Manually set this value with unsorted or duplicated values at your own risk. Duplicated values are unlikely to + * cause a problem, but unsorted values can result in incorrect results. + */ + private final Supplier> sortedMatchValues; + + /** + * Supplier for list of utf8 byte values sorted by {@link #matchValueType}. If {@link #sortedMatchValues} was supplied + * directly instead of lazily computed, and {@link #matchValueType} is {@link ColumnType#STRING}, the backing list + * will be eagerly computed. If {@link #sortedMatchValues} is lazily computed, this value will be null. + */ + @Nullable + private final Supplier> sortedUtf8MatchValueBytes; + @Nullable + private final FilterTuning filterTuning; + private final Supplier predicateFactorySupplier; + @JsonIgnore + private final Supplier cacheKeySupplier; + + /** + * Creates a new filter. + * + * @param column column to search + * @param values set of values to match, may or may not be sorted. + * @param sortedValues set of values to match this is sorted in matchValueType order. These values absolutely must + * be sorted in the specified order for proper operation. This value is computed from values to + * be used 'downstream' to avoid repeating the work of sorting and checking for sortedness over + * and over. + * @param matchValueType type of values contained in values/sortedValues + * @param filterTuning optional tuning + */ + @JsonCreator + public TypedInFilter( + @JsonProperty("column") String column, + @JsonProperty("matchValueType") ColumnType matchValueType, + @JsonProperty("values") @Nullable List values, + @JsonProperty("sortedValues") @Nullable List sortedValues, + @JsonProperty("filterTuning") @Nullable FilterTuning filterTuning + ) + { + this.column = column; + if (column == null) { + throw InvalidInput.exception("Invalid IN filter, column cannot be null"); + } + this.filterTuning = filterTuning; + this.matchValueType = matchValueType; + if (matchValueType == null) { + throw InvalidInput.exception("Invalid IN filter on column [%s], matchValueType cannot be null", column); + } + // one of sorted or not sorted + if ((values == null && sortedValues == null) || (values != null && sortedValues != null)) { + throw InvalidInput.exception( + "Invalid IN filter on column [%s], exactly one of values or sortedValues must be non-null", + column + ); + } + if (sortedValues != null) { + this.unsortedValues = null; + this.sortedMatchValues = () -> sortedValues; + if (matchValueType.is(ValueType.STRING)) { + final List matchValueBytes = Lists.newArrayListWithCapacity(sortedValues.size()); + for (Object s : sortedMatchValues.get()) { + matchValueBytes.add(StringUtils.toUtf8ByteBuffer(Evals.asString(s))); + } + this.sortedUtf8MatchValueBytes = () -> matchValueBytes; + } else { + this.sortedUtf8MatchValueBytes = null; + } + } else { + if (checkSorted(values, matchValueType)) { + this.unsortedValues = null; + this.sortedMatchValues = () -> values; + } else { + this.unsortedValues = values; + this.sortedMatchValues = Suppliers.memoize(() -> sortValues(unsortedValues, matchValueType)); + } + this.sortedUtf8MatchValueBytes = null; + } + + this.predicateFactorySupplier = Suppliers.memoize( + () -> new PredicateFactory(sortedMatchValues.get(), matchValueType) + ); + this.cacheKeySupplier = Suppliers.memoize(this::computeCacheKey); + } + + @JsonProperty + public String getColumn() + { + return column; + } + + @JsonProperty + public List getSortedValues() + { + return sortedMatchValues.get(); + } + + @JsonProperty + public ColumnType getMatchValueType() + { + return matchValueType; + } + + @Nullable + @JsonInclude(JsonInclude.Include.NON_NULL) + @JsonProperty + public FilterTuning getFilterTuning() + { + return filterTuning; + } + + @Override + public byte[] getCacheKey() + { + return cacheKeySupplier.get(); + } + + @Override + public DimFilter optimize(final boolean mayIncludeUnknown) + { + checkSqlCompatible(); + final List matchValues = this.sortedMatchValues.get(); + if (matchValues.isEmpty()) { + return FalseDimFilter.instance(); + } else if (matchValues.size() == 1) { + if (matchValues.get(0) == null) { + return NullFilter.forColumn(column); + } + return new EqualityFilter( + column, + matchValueType, + matchValues.iterator().next(), + filterTuning + ); + } + return this; + } + + @Override + public Filter toFilter() + { + checkSqlCompatible(); + return this; + } + + @Nullable + @Override + public RangeSet getDimensionRangeSet(String dimension) + { + if (!Objects.equals(getColumn(), dimension)) { + return null; + } + RangeSet retSet = TreeRangeSet.create(); + for (Object value : sortedMatchValues.get()) { + String valueEquivalent = Evals.asString(value); + if (valueEquivalent == null) { + // Range.singleton(null) is invalid, so use the fact that + // only null values are less than empty string. + retSet.add(Range.lessThan("")); + } else { + retSet.add(Range.singleton(valueEquivalent)); + } + } + return retSet; + } + + @Override + public Set getRequiredColumns() + { + return ImmutableSet.of(column); + } + + @Override + @Nullable + public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) + { + if (!Filters.checkFilterTuningUseIndex(column, selector, filterTuning)) { + return null; + } + final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column); + + if (indexSupplier == null) { + // column doesn't exist, match against null + DruidPredicateMatch match = predicateFactorySupplier.get().makeStringPredicate().apply(null); + return Filters.makeMissingColumnNullIndex(match, selector); + } + + if (sortedUtf8MatchValueBytes != null) { + final Utf8ValueSetIndexes utf8ValueSetIndexes = indexSupplier.as(Utf8ValueSetIndexes.class); + if (utf8ValueSetIndexes != null) { + return utf8ValueSetIndexes.forSortedValuesUtf8(sortedUtf8MatchValueBytes.get()); + } + } + + final ValueSetIndexes valueSetIndexes = indexSupplier.as(ValueSetIndexes.class); + if (valueSetIndexes != null) { + return valueSetIndexes.forSortedValues(sortedMatchValues.get(), matchValueType); + } + + return Filters.makePredicateIndex( + column, + selector, + predicateFactorySupplier.get() + ); + } + + @Override + public ValueMatcher makeMatcher(ColumnSelectorFactory factory) + { + return Filters.makeValueMatcher(factory, column, predicateFactorySupplier.get()); + } + + @Override + public VectorValueMatcher makeVectorMatcher(final VectorColumnSelectorFactory factory) + { + return ColumnProcessors.makeVectorProcessor( + column, + VectorValueMatcherColumnProcessorFactory.instance(), + factory + ).makeMatcher(predicateFactorySupplier.get()); + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return true; + } + + @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); + } + + if (rewriteDimensionTo.equals(column)) { + return this; + } else { + return new TypedInFilter( + rewriteDimensionTo, + matchValueType, + null, + sortedMatchValues.get(), + filterTuning + ); + } + } + + @Override + public String toString() + { + final DimFilter.DimFilterToStringBuilder builder = new DimFilter.DimFilterToStringBuilder(); + return builder.appendDimension(column, null) + .append(" IN (") + .append(Joiner.on(", ").join(Iterables.transform(sortedMatchValues.get(), String::valueOf))) + .append(")") + .append(" (" + matchValueType + ")") + .appendFilterTuning(filterTuning) + .build(); + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TypedInFilter that = (TypedInFilter) o; + return column.equals(that.column) && + Objects.equals(matchValueType, that.matchValueType) && + compareValues(sortedMatchValues.get(), that.sortedMatchValues.get(), matchValueType) && + Objects.equals(filterTuning, that.filterTuning); + } + + @Override + public int hashCode() + { + return Objects.hash(sortedMatchValues.get(), column, matchValueType, filterTuning); + } + + private byte[] computeCacheKey() + { + // Hash all values, in sorted order, as their length followed by their content. + final Hasher hasher = Hashing.sha256().newHasher(); + for (Object v : sortedMatchValues.get()) { + if (v == null) { + // Encode null as length -1, no content. + hasher.putInt(-1); + } else { + final String s = Evals.asString(v); + hasher.putInt(s.length()); + hasher.putString(s, StandardCharsets.UTF_8); + } + } + + return new CacheKeyBuilder(DimFilterUtils.TYPED_IN_CACHE_ID) + .appendString(column) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendString(matchValueType.asTypeString()) + .appendByte(DimFilterUtils.STRING_SEPARATOR) + .appendByteArray(hasher.hash().asBytes()) + .build(); + } + + private void checkSqlCompatible() + { + if (NullHandling.replaceWithDefault()) { + throw InvalidInput.exception( + "Invalid IN filter, typed in filter only supports SQL compatible null handling mode, set druid.generic.useDefaultValue=false to use this filter" + ); + } + } + + private static boolean checkSorted(List unsortedValues, ColumnType matchValueType) + { + final Comparator comparator = matchValueType.getNullableStrategy(); + Object prev = null; + for (Object o : unsortedValues) { + if (o != null) { + Object coerced = coerceValue(o, matchValueType); + //noinspection ObjectEquality + if (coerced != o) { + return false; + } + } + if (prev != null && comparator.compare(prev, o) >= 0) { + return false; + } + prev = o; + } + return true; + } + + @Nullable + private static Object coerceValue(@Nullable Object o, ColumnType matchValueType) + { + if (o == null) { + return null; + } + switch (matchValueType.getType()) { + case STRING: + return DimensionHandlerUtils.convertObjectToString(o); + case LONG: + return DimensionHandlerUtils.convertObjectToLong(o); + case FLOAT: + return DimensionHandlerUtils.convertObjectToFloat(o); + case DOUBLE: + return DimensionHandlerUtils.convertObjectToDouble(o); + default: + throw InvalidInput.exception("Unsupported matchValueType[%s]", matchValueType); + } + } + + private static List sortValues(List unsortedValues, ColumnType matchValueType) + { + final Object[] array = unsortedValues.toArray(new Object[0]); + // coerce values to matchValueType + for (int i = 0; i < array.length; i++) { + Object coerced = coerceValue(array[i], matchValueType); + array[i] = coerced; + } + final Comparator comparator = matchValueType.getNullableStrategy(); + ObjectArrays.quickSort(array, comparator); + // dedupe values + final List sortedList = Lists.newArrayListWithCapacity(array.length); + for (int i = 0; i < array.length; i++) { + if (i > 0 && comparator.compare(array[i - 1], array[i]) == 0) { + continue; + } + sortedList.add(array[i]); + } + return sortedList; + } + + /** + * Since jackson might translate longs into ints and such, we use the type comparator to check lists + * for {@link #equals(Object)} instead of directly using {@link Objects#equals(Object, Object)} + */ + private static boolean compareValues(List o1, List o2, ColumnType matchValueType) + { + final NullableTypeStrategy comparator = matchValueType.getNullableStrategy(); + //noinspection ObjectEquality + if (o1 == o2) { + return true; + } + if (o1 == null) { + return false; + } + if (o2 == null) { + return false; + } + final int size1 = o1.size(); + final int size2 = o2.size(); + if (size1 != size2) { + return false; + } + for (int i = 0; i < size1; i++) { + final int cmp = comparator.compare(o1.get(i), o2.get(i)); + if (cmp == 0) { + continue; + } + return false; + } + return true; + } + + private static DruidObjectPredicate createStringPredicate( + final List sortedValues, + final ColumnType matchValueType + ) + { + Preconditions.checkNotNull(sortedValues, "values"); + final boolean containsNull = !sortedValues.isEmpty() && sortedValues.get(0) == null; + final Comparator comparator = matchValueType.getNullableStrategy(); + if (matchValueType.is(ValueType.STRING)) { + return value -> { + if (value == null) { + return containsNull ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + final int index = Collections.binarySearch(sortedValues, value, comparator); + return DruidPredicateMatch.of(index >= 0); + }; + } + // convert set to strings + final Set stringSet = Sets.newHashSetWithExpectedSize(sortedValues.size()); + for (Object o : sortedValues) { + stringSet.add(Evals.asString(o)); + } + return value -> { + if (value == null) { + return containsNull ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + return DruidPredicateMatch.of(stringSet.contains(value)); + }; + } + + private static DruidLongPredicate createLongPredicate(final List sortedValues, ColumnType matchValueType) + { + boolean matchNulls = !sortedValues.isEmpty() && sortedValues.get(0) == null; + if (matchValueType.is(ValueType.LONG)) { + final Comparator comparator = matchValueType.getNullableStrategy(); + return new DruidLongPredicate() + { + @Override + public DruidPredicateMatch applyLong(long input) + { + final int index = Collections.binarySearch(sortedValues, input, comparator); + return DruidPredicateMatch.of(index >= 0); + } + + @Override + public DruidPredicateMatch applyNull() + { + return matchNulls ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + }; + } + // convert set to longs + LongOpenHashSet longs = new LongOpenHashSet(); + for (Object value : sortedValues) { + final Long longValue = DimensionHandlerUtils.convertObjectToLong(value); + if (longValue != null) { + longs.add(longValue.longValue()); + } + } + return new DruidLongPredicate() + { + @Override + public DruidPredicateMatch applyLong(long input) + { + return DruidPredicateMatch.of(longs.contains(input)); + } + + @Override + public DruidPredicateMatch applyNull() + { + return matchNulls ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + }; + } + + private static DruidFloatPredicate createFloatPredicate(final List sortedValues, ColumnType matchValueType) + { + boolean matchNulls = !sortedValues.isEmpty() && sortedValues.get(0) == null; + if (matchValueType.is(ValueType.FLOAT)) { + final Comparator comparator = matchValueType.getNullableStrategy(); + return new DruidFloatPredicate() + { + @Override + public DruidPredicateMatch applyFloat(float input) + { + final int index = Collections.binarySearch(sortedValues, input, comparator); + return DruidPredicateMatch.of(index >= 0); + } + + @Override + public DruidPredicateMatch applyNull() + { + return matchNulls ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + }; + } + // convert set to floats + final FloatOpenHashSet floatSet = new FloatOpenHashSet(); + for (Object value : sortedValues) { + final Float floatValue = DimensionHandlerUtils.convertObjectToFloat(value); + if (floatValue != null) { + floatSet.add(floatValue.floatValue()); + } + } + return new DruidFloatPredicate() + { + @Override + public DruidPredicateMatch applyFloat(float input) + { + return DruidPredicateMatch.of(floatSet.contains(input)); + } + + @Override + public DruidPredicateMatch applyNull() + { + return matchNulls ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + }; + } + + private static DruidDoublePredicate createDoublePredicate(final List sortedValues, ColumnType matchValueType) + { + boolean matchNulls = !sortedValues.isEmpty() && sortedValues.get(0) == null; + if (matchValueType.is(ValueType.DOUBLE)) { + final Comparator comparator = matchValueType.getNullableStrategy(); + return new DruidDoublePredicate() + { + @Override + public DruidPredicateMatch applyDouble(double input) + { + final int index = Collections.binarySearch(sortedValues, input, comparator); + return DruidPredicateMatch.of(index >= 0); + } + + @Override + public DruidPredicateMatch applyNull() + { + return matchNulls ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + }; + } + + // convert set to doubles + final DoubleOpenHashSet doubleSet = new DoubleOpenHashSet(sortedValues.size()); + for (Object value : sortedValues) { + Double doubleValue = DimensionHandlerUtils.convertObjectToDouble(value); + if (doubleValue != null) { + doubleSet.add(doubleValue.doubleValue()); + } + } + return new DruidDoublePredicate() + { + @Override + public DruidPredicateMatch applyDouble(double input) + { + return DruidPredicateMatch.of(doubleSet.contains(input)); + } + + @Override + public DruidPredicateMatch applyNull() + { + return matchNulls ? DruidPredicateMatch.TRUE : DruidPredicateMatch.UNKNOWN; + } + }; + } + + public static class PredicateFactory implements DruidPredicateFactory + { + private final ColumnType matchValueType; + private final List sortedValues; + private final Supplier> stringPredicateSupplier; + private final Supplier longPredicateSupplier; + private final Supplier floatPredicateSupplier; + private final Supplier doublePredicateSupplier; + + public PredicateFactory(final List sortedValues, final ColumnType matchValueType) + { + this.sortedValues = sortedValues; + this.matchValueType = matchValueType; + + // As the set of filtered values can be large, parsing them as numbers should be done only if needed, and + // only once. Pass in a common long predicate supplier to all filters created by .toFilter(), so that we only + // compute the long hashset/array once per query. This supplier must be thread-safe, since this DimFilter will be + // accessed in the query runners. + this.stringPredicateSupplier = Suppliers.memoize(() -> createStringPredicate(sortedValues, matchValueType)); + this.longPredicateSupplier = Suppliers.memoize(() -> createLongPredicate(sortedValues, matchValueType)); + this.floatPredicateSupplier = Suppliers.memoize(() -> createFloatPredicate(sortedValues, matchValueType)); + this.doublePredicateSupplier = Suppliers.memoize(() -> createDoublePredicate(sortedValues, matchValueType)); + } + + @Override + public DruidObjectPredicate 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 boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PredicateFactory that = (PredicateFactory) o; + return Objects.equals(matchValueType, that.matchValueType) && + Objects.equals(sortedValues, that.sortedValues); + } + + @Override + public int hashCode() + { + return Objects.hash(matchValueType, sortedValues); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java index 1421e4fc16ef..154dd66e206e 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java @@ -27,7 +27,9 @@ /** * Common interface for bitmap indexes for use by {@link org.apache.druid.query.filter.Filter} for cursor creation, to - * allow fast row skipping during query processing. + * allow fast row skipping during query processing. Ideally implementaitons of this are 'lazy', and not do any real + * work until {@link #computeBitmapResult(BitmapResultFactory, int, int, boolean)} or + * {@link #computeBitmapResult(BitmapResultFactory, boolean)} is called. */ public interface BitmapColumnIndex { diff --git a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java index 8ab99f05787f..65395d148b26 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java @@ -20,10 +20,9 @@ package org.apache.druid.segment.index; import com.google.common.base.Preconditions; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; -import com.google.common.collect.Iterators; -import com.google.common.collect.PeekingIterator; import org.apache.druid.annotations.SuppressFBWarnings; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; @@ -33,23 +32,25 @@ 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.DimensionHandlerUtils; 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 org.apache.druid.segment.index.semantic.ValueSetIndexes; import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.Comparator; -import java.util.Iterator; -import java.util.NoSuchElementException; +import java.util.List; import java.util.SortedSet; public final class IndexedUtf8ValueIndexes> - implements StringValueSetIndexes, Utf8ValueSetIndexes, ValueIndexes + implements StringValueSetIndexes, Utf8ValueSetIndexes, ValueIndexes, ValueSetIndexes { // 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 @@ -132,18 +133,27 @@ public BitmapColumnIndex forSortedValues(SortedSet values) @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION") @Override - public BitmapColumnIndex forSortedValuesUtf8(SortedSet valuesUtf8) + public BitmapColumnIndex forSortedValuesUtf8(List sortedValuesUtf8) { - final SortedSet tailSet; + if (sortedValuesUtf8.isEmpty()) { + return new AllFalseBitmapColumnIndex(bitmapFactory); + } + final boolean matchNull = sortedValuesUtf8.get(0) == null; + final List tailSet; - if (valuesUtf8.size() >= SIZE_WORTH_CHECKING_MIN) { + if (sortedValuesUtf8.size() >= SIZE_WORTH_CHECKING_MIN) { final ByteBuffer minValueInColumn = dictionary.get(0); - tailSet = valuesUtf8.tailSet(minValueInColumn); + final int position = Collections.binarySearch( + sortedValuesUtf8, + minValueInColumn, + ByteBufferUtils.utf8Comparator() + ); + tailSet = sortedValuesUtf8.subList(position >= 0 ? position : -(position + 1), sortedValuesUtf8.size()); } else { - tailSet = valuesUtf8; + tailSet = sortedValuesUtf8; } - return getBitmapColumnIndexForSortedIterableUtf8(tailSet, tailSet.size(), valuesUtf8.contains(null)); + return getBitmapColumnIndexForSortedIterableUtf8(tailSet, tailSet.size(), matchNull); } private ImmutableBitmap getBitmap(int idx) @@ -159,149 +169,103 @@ private ImmutableBitmap getBitmap(int idx) /** * Helper used by {@link #forSortedValues} and {@link #forSortedValuesUtf8}. */ - private BitmapColumnIndex getBitmapColumnIndexForSortedIterableUtf8(Iterable valuesUtf8, int size, boolean valuesContainsNull) + private BitmapColumnIndex getBitmapColumnIndexForSortedIterableUtf8( + Iterable valuesUtf8, + int size, + boolean valuesContainsNull + ) { // for large number of in-filter values in comparison to the dictionary size, use the sorted merge algorithm. if (size > SORTED_MERGE_RATIO_THRESHOLD * dictionary.size()) { - return new SimpleImmutableBitmapDelegatingIterableIndex() - { - @Override - public Iterable getBitmapIterable() - { - return () -> new Iterator() - { - final PeekingIterator valuesIterator = Iterators.peekingIterator(valuesUtf8.iterator()); - final PeekingIterator dictionaryIterator = Iterators.peekingIterator(dictionary.iterator()); - int next = -1; - int idx = 0; - - @Override - public boolean hasNext() - { - if (next < 0) { - findNext(); - } - return next >= 0; - } - - @Override - public ImmutableBitmap next() - { - if (next < 0) { - findNext(); - if (next < 0) { - throw new NoSuchElementException(); - } - } - final int swap = next; - next = -1; - return getBitmap(swap); - } - - private void findNext() - { - while (next < 0 && valuesIterator.hasNext() && dictionaryIterator.hasNext()) { - final ByteBuffer nextValue = valuesIterator.peek(); - final ByteBuffer nextDictionaryKey = dictionaryIterator.peek(); - final int comparison = COMPARATOR.compare(nextValue, nextDictionaryKey); - if (comparison == 0) { - next = idx; - valuesIterator.next(); - break; - } else if (comparison < 0) { - valuesIterator.next(); - } else { - dictionaryIterator.next(); - idx++; - } - } + return ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorScan( + bitmapFactory, + COMPARATOR, + valuesUtf8, + dictionary, + bitmaps, + () -> { + if (!valuesContainsNull && NullHandling.isNullOrEquivalent(dictionary.get(0))) { + return bitmaps.get(0); } - }; - } + return null; + } + ); + } - @Nullable - @Override - protected ImmutableBitmap getUnknownsBitmap() - { + // 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 ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorBinarySearch( + bitmapFactory, + valuesUtf8, + dictionary, + bitmaps, + () -> { if (!valuesContainsNull && NullHandling.isNullOrEquivalent(dictionary.get(0))) { return bitmaps.get(0); } return null; } - }; - } - - // 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, valuesContainsNull); + ); } - /** - * 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 SimpleImmutableBitmapDelegatingIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable valuesUtf8, boolean valuesContainsNull) + @Nullable + @Override + public BitmapColumnIndex forSortedValues(@Nonnull List sortedValues, TypeSignature matchValueType) { - return new SimpleImmutableBitmapDelegatingIterableIndex() - { - @Override - public Iterable getBitmapIterable() - { - return () -> new Iterator() - { - final int dictionarySize = dictionary.size(); - final Iterator iterator = valuesUtf8.iterator(); - int next = -1; - - @Override - public boolean hasNext() - { - if (next < 0) { - findNext(); - } - return next >= 0; - } - - @Override - public ImmutableBitmap next() - { - if (next < 0) { - findNext(); - if (next < 0) { - throw new NoSuchElementException(); - } - } - final int swap = next; - next = -1; - return getBitmap(swap); - } - - private void findNext() - { - while (next < 0 && iterator.hasNext()) { - ByteBuffer nextValue = iterator.next(); - next = dictionary.indexOf(nextValue); - - if (next == -dictionarySize - 1) { - // nextValue is past the end of the dictionary so we can break early - // Note: we can rely on indexOf returning (-(insertion point) - 1), because of the earlier check - // for Indexed.isSorted(), which guarantees this behavior - break; - } - } - } - }; + if (sortedValues.isEmpty()) { + return new AllFalseBitmapColumnIndex(bitmapFactory); + } + final boolean matchNull = sortedValues.get(0) == null; + final Supplier unknownsIndex = () -> { + if (!matchNull && dictionary.get(0) == null) { + return bitmaps.get(0); } + return null; + }; + if (matchValueType.is(ValueType.STRING)) { + final List tailSet; + final List baseSet = (List) sortedValues; - @Nullable - @Override - protected ImmutableBitmap getUnknownsBitmap() - { - if (!valuesContainsNull && NullHandling.isNullOrEquivalent(dictionary.get(0))) { - return bitmaps.get(0); - } - return null; + if (sortedValues.size() >= ValueSetIndexes.SIZE_WORTH_CHECKING_MIN) { + final Object minValueInColumn = dictionary.get(0); + final int position = Collections.binarySearch( + sortedValues, + StringUtils.fromUtf8((ByteBuffer) minValueInColumn), + matchValueType.getNullableStrategy() + ); + tailSet = baseSet.subList(position >= 0 ? position : -(position + 1), baseSet.size()); + } else { + tailSet = baseSet; } - }; + if (tailSet.size() > ValueSetIndexes.SORTED_SCAN_RATIO_THRESHOLD * dictionary.size()) { + return ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorScan( + bitmapFactory, + ByteBufferUtils.utf8Comparator(), + Iterables.transform(tailSet, StringUtils::toUtf8ByteBuffer), + dictionary, + bitmaps, + unknownsIndex + ); + } + // fall through to value iteration + return ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorBinarySearch( + bitmapFactory, + Iterables.transform(tailSet, StringUtils::toUtf8ByteBuffer), + dictionary, + bitmaps, + unknownsIndex + ); + } else { + return ValueSetIndexes.buildBitmapColumnIndexFromIteratorBinarySearch( + bitmapFactory, + Iterables.transform( + sortedValues, + x -> StringUtils.toUtf8ByteBuffer(DimensionHandlerUtils.convertObjectToString(x)) + ), + dictionary, + bitmaps, + unknownsIndex + ); + } } } diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/ArrayElementIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/ArrayElementIndexes.java index 5a8a9d841b55..90eb8ce8a8f0 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/semantic/ArrayElementIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/ArrayElementIndexes.java @@ -26,6 +26,10 @@ import javax.annotation.Nullable; +/** + * Construct a {@link BitmapColumnIndex} for any array element which might be present in an array contained in the + * column. + */ public interface ArrayElementIndexes { /** diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedValueIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedValueIndex.java index a99f91c2dc1d..b60115e4066a 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedValueIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/DictionaryEncodedValueIndex.java @@ -54,6 +54,5 @@ public interface DictionaryEncodedValueIndex @Nullable T getValue(int index); - @SuppressWarnings({"unreachable", "unused"}) BitmapFactory getBitmapFactory(); } diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/DruidPredicateIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/DruidPredicateIndexes.java index c7e87ec20f1e..f0e802f7fab3 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/semantic/DruidPredicateIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/DruidPredicateIndexes.java @@ -25,7 +25,8 @@ import javax.annotation.Nullable; /** - * Uses a {@link DruidPredicateFactory} to construct a {@link BitmapColumnIndex} + * Uses a {@link DruidPredicateFactory} to construct a {@link BitmapColumnIndex} containing rows for all values which + * satisfy the predicate. */ public interface DruidPredicateIndexes { diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/NullValueIndex.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/NullValueIndex.java index 8768caa54627..884866f95769 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/semantic/NullValueIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/NullValueIndex.java @@ -22,7 +22,7 @@ import org.apache.druid.segment.index.BitmapColumnIndex; /** - * Provides index for all null rows in a column, to use with IS/IS NOT NULL filters + * Provides {@link BitmapColumnIndex} for all null rows in a column, to use with IS/IS NOT NULL filters */ public interface NullValueIndex { diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/StringValueSetIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/StringValueSetIndexes.java index acb4b6712716..09c88776e9d4 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/semantic/StringValueSetIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/StringValueSetIndexes.java @@ -27,18 +27,20 @@ /** * Index on individual values, and provides bitmaps for the rows which contain these values + * @deprecated use {@link ValueIndexes}, {@link ValueSetIndexes}, or {@link Utf8ValueSetIndexes} instead */ +@Deprecated public interface StringValueSetIndexes { /** - * Get the {@link ImmutableBitmap} corresponding to the supplied value. Generates an empty bitmap when passed a - * value that doesn't exist. Never returns null. + * Get the wrapped {@link ImmutableBitmap} corresponding to the supplied value. Generates an empty bitmap when + * passed a value that doesn't exist. Never returns null. */ BitmapColumnIndex forValue(@Nullable String value); /** - * Get an {@link Iterable} of {@link ImmutableBitmap} corresponding to the specified set of values (if they are - * contained in the underlying column). The set must be sorted using + * Get the wrapped {@link ImmutableBitmap} corresponding to the specified set of values (if they are contained in the + * underlying column). The set must be sorted using * {@link org.apache.druid.java.util.common.guava.Comparators#naturalNullsFirst()}. */ BitmapColumnIndex forSortedValues(SortedSet values); diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/Utf8ValueSetIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/Utf8ValueSetIndexes.java index 1bf2792e444d..2180c76d30f1 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/semantic/Utf8ValueSetIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/Utf8ValueSetIndexes.java @@ -23,14 +23,17 @@ import org.apache.druid.segment.index.BitmapColumnIndex; import java.nio.ByteBuffer; -import java.util.SortedSet; +import java.util.List; +/** + * Construct a {@link BitmapColumnIndex} for a set of utf8 byte values which might be present in the column. + */ public interface Utf8ValueSetIndexes { /** - * Get an {@link Iterable} of {@link ImmutableBitmap} corresponding to the specified set of values (if they are - * contained in the underlying column). The set must be sorted using + * Get the wrapped {@link ImmutableBitmap} corresponding to the specified set of values (if they are contained in the + * underlying column). The set must be sorted using * {@link org.apache.druid.java.util.common.ByteBufferUtils#utf8Comparator()}. */ - BitmapColumnIndex forSortedValuesUtf8(SortedSet valuesUtf8); + BitmapColumnIndex forSortedValuesUtf8(List sortedValuesUtf8); } 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 index 9eee56896d95..4234a7b47bec 100644 --- 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 @@ -27,13 +27,17 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +/** + * Construct a {@link BitmapColumnIndex} for a specific value which might be present in the column. + */ public interface ValueIndexes { /** - * Get the {@link ImmutableBitmap} corresponding to rows matching 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. - * + * Get a {@link BitmapColumnIndex} which can compute the {@link ImmutableBitmap} corresponding to rows matching 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. + *

* Does not match null, use {@link NullValueIndex} for matching nulls. * * @param value value to match diff --git a/processing/src/main/java/org/apache/druid/segment/index/semantic/ValueSetIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/semantic/ValueSetIndexes.java new file mode 100644 index 000000000000..56d6da9a19af --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/index/semantic/ValueSetIndexes.java @@ -0,0 +1,331 @@ +/* + * 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 com.google.common.base.Supplier; +import com.google.common.collect.Iterators; +import com.google.common.collect.PeekingIterator; +import org.apache.druid.collections.bitmap.BitmapFactory; +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.data.Indexed; +import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.SimpleImmutableBitmapDelegatingIterableIndex; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Construct a {@link BitmapColumnIndex} for a set of values which might be present in the column. + */ +public interface ValueSetIndexes +{ + /** + * threshold of sorted match value iterator size compared to dictionary size to use + * {@link #buildBitmapColumnIndexFromSortedIteratorScan} instead of + * {@link #buildBitmapColumnIndexFromSortedIteratorBinarySearch}. + */ + double SORTED_SCAN_RATIO_THRESHOLD = 0.12D; + + /** + * minimum sorted match value iterator size to trim the initial values from the iterator to seek to the start of the + * value dictionary when using {@link #buildBitmapColumnIndexFromSortedIteratorScan} or + * {@link #buildBitmapColumnIndexFromSortedIteratorBinarySearch}. + */ + int SIZE_WORTH_CHECKING_MIN = 8; + + /** + * Get the wrapped {@link ImmutableBitmap} corresponding to the specified set of values (if they are contained in the + * underlying column). The set must be sorted using the comparator of the supplied matchValueType. + * + * @param sortedValues values to match, sorted in matchValueType order + * @param matchValueType 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 values, or null if an index + * connot be computed for the supplied value type + */ + @Nullable + BitmapColumnIndex forSortedValues(@Nonnull List sortedValues, TypeSignature matchValueType); + + + /** + * Helper method for implementing {@link #forSortedValues} for a value set that is sorted the same as the column + * dictionary. + *

+ * Builds a {@link BitmapColumnIndex} from an {@link Iterable} that is sorted the same as the columns + * {@link Indexed} value dictionary. Uses a strategy that does zipping similar to the merge step of a sort-merge, + * where we step forward on both the iterator and the dictionary to find matches to build a + * {@link Iterable}. + *

+ * If sorted match value iterator size is greater than (dictionary size * {@link #SORTED_SCAN_RATIO_THRESHOLD}), + * consider using this method instead of {@link #buildBitmapColumnIndexFromSortedIteratorBinarySearch}. + *

+ * If the values in the iterator are NOT sorted the same as the dictionary, do NOT use this method, use + * {@link #buildBitmapColumnIndexFromIteratorBinarySearch} instead. + */ + static BitmapColumnIndex buildBitmapColumnIndexFromSortedIteratorScan( + BitmapFactory bitmapFactory, + Comparator comparator, + Iterable values, + Indexed dictionary, + Indexed bitmaps, + Supplier unknownsBitmap + ) + { + return new BaseValueSetIndexesFromIterable(bitmapFactory, bitmaps, unknownsBitmap) + { + @Override + public Iterable getBitmapIterable() + { + return () -> new Iterator() + { + final PeekingIterator valuesIterator = Iterators.peekingIterator(values.iterator()); + final PeekingIterator dictionaryIterator = Iterators.peekingIterator(dictionary.iterator()); + int next = -1; + int idx = 0; + + @Override + public boolean hasNext() + { + if (next < 0) { + findNext(); + } + return next >= 0; + } + + @Override + public ImmutableBitmap next() + { + if (next < 0) { + findNext(); + if (next < 0) { + throw new NoSuchElementException(); + } + } + final int swap = next; + next = -1; + return getBitmap(swap); + } + + private void findNext() + { + while (next < 0 && valuesIterator.hasNext() && dictionaryIterator.hasNext()) { + final T nextValue = valuesIterator.peek(); + final T nextDictionaryKey = dictionaryIterator.peek(); + final int comparison = comparator.compare(nextValue, nextDictionaryKey); + if (comparison == 0) { + next = idx; + valuesIterator.next(); + break; + } else if (comparison < 0) { + valuesIterator.next(); + } else { + dictionaryIterator.next(); + idx++; + } + } + } + }; + } + }; + } + + /** + * Helper method for implementing {@link #forSortedValues} for a value set that is sorted the same as the column + * dictionary. + *

+ * Builds a {@link BitmapColumnIndex} from an {@link Iterable} that is sorted the same as the columns + * {@link Indexed} value dictionary. This algorithm iterates the values to match and does a binary search for + * matching values using {@link Indexed#indexOf(Object)} to build a {@link Iterable} short-circuiting + * the iteration if we reach the end of the {@link Indexed} before the values to match are exhausted. + *

+ * If sorted match value iterator size is less than (dictionary size * {@link #SORTED_SCAN_RATIO_THRESHOLD}), + * consider using this method instead of {@link #buildBitmapColumnIndexFromSortedIteratorScan}. + *

+ * If the values in the iterator are not sorted the same as the dictionary, do not use this method, use + * {@link #buildBitmapColumnIndexFromIteratorBinarySearch} instead. + */ + static BitmapColumnIndex buildBitmapColumnIndexFromSortedIteratorBinarySearch( + BitmapFactory bitmapFactory, + Iterable values, + Indexed dictionary, + Indexed bitmaps, + Supplier getUnknownsIndex + ) + { + return new BaseValueSetIndexesFromIterable(bitmapFactory, bitmaps, getUnknownsIndex) + { + @Override + public Iterable getBitmapIterable() + { + return () -> new Iterator() + { + final int dictionarySize = dictionary.size(); + final Iterator iterator = values.iterator(); + int next = -1; + + @Override + public boolean hasNext() + { + if (next < 0) { + findNext(); + } + return next >= 0; + } + + @Override + public ImmutableBitmap next() + { + if (next < 0) { + findNext(); + if (next < 0) { + throw new NoSuchElementException(); + } + } + final int swap = next; + next = -1; + return getBitmap(swap); + } + + private void findNext() + { + while (next < 0 && iterator.hasNext()) { + T nextValue = iterator.next(); + next = dictionary.indexOf(nextValue); + + if (next == -dictionarySize - 1) { + // nextValue is past the end of the dictionary so we can break early + // Note: we can rely on indexOf returning (-(insertion point) - 1), because the Indexed + // is sorted, which guarantees this behavior + break; + } + } + } + }; + } + }; + } + + /** + * Helper method for implementing {@link #forSortedValues} for a value set that is NOT sorted the same as the column + * dictionary. + *

+ * Builds a {@link BitmapColumnIndex} from an {@link Iterable} that is NOT sorted the same as the columns + * {@link Indexed} value dictionary. This algorithm iterates the values to match and does a binary search for + * matching values using {@link Indexed#indexOf(Object)} to build a {@link Iterable} until the match + * values iterator is exhausted. + *

+ * If values of the iterator are sorted the same as the dictionary, use + * {@link #buildBitmapColumnIndexFromSortedIteratorScan} or + * {@link #buildBitmapColumnIndexFromSortedIteratorBinarySearch} instead. + */ + static BitmapColumnIndex buildBitmapColumnIndexFromIteratorBinarySearch( + BitmapFactory bitmapFactory, + Iterable values, + Indexed dictionary, + Indexed bitmaps, + Supplier getUnknownsIndex + ) + { + return new BaseValueSetIndexesFromIterable(bitmapFactory, bitmaps, getUnknownsIndex) + { + @Override + public Iterable getBitmapIterable() + { + return () -> new Iterator() + { + final Iterator iterator = values.iterator(); + int next = -1; + + @Override + public boolean hasNext() + { + if (next < 0) { + findNext(); + } + return next >= 0; + } + + @Override + public ImmutableBitmap next() + { + if (next < 0) { + findNext(); + if (next < 0) { + throw new NoSuchElementException(); + } + } + final int swap = next; + next = -1; + return getBitmap(swap); + } + + private void findNext() + { + while (next < 0 && iterator.hasNext()) { + T nextValue = iterator.next(); + next = dictionary.indexOf(nextValue); + } + } + }; + } + }; + } + + abstract class BaseValueSetIndexesFromIterable extends SimpleImmutableBitmapDelegatingIterableIndex + { + private final Indexed bitmaps; + private final BitmapFactory bitmapFactory; + private final Supplier unknownsBitmap; + + public BaseValueSetIndexesFromIterable( + BitmapFactory bitmapFactory, + Indexed bitmaps, + Supplier unknownsBitmap + ) + { + this.bitmaps = bitmaps; + this.bitmapFactory = bitmapFactory; + this.unknownsBitmap = unknownsBitmap; + } + + @Nullable + @Override + protected ImmutableBitmap getUnknownsBitmap() + { + return unknownsBitmap.get(); + } + + protected ImmutableBitmap getBitmap(int idx) + { + if (idx < 0) { + return bitmapFactory.makeEmptyImmutableBitmap(); + } + + final ImmutableBitmap bitmap = bitmaps.get(idx); + return bitmap == null ? bitmapFactory.makeEmptyImmutableBitmap() : bitmap; + } + } +} 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 97788148288c..ff0019a2b687 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 @@ -22,6 +22,7 @@ import com.google.common.base.Strings; import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.primitives.Doubles; import it.unimi.dsi.fastutil.doubles.DoubleArraySet; import it.unimi.dsi.fastutil.doubles.DoubleIterator; @@ -40,6 +41,7 @@ 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.DimensionHandlerUtils; import org.apache.druid.segment.IntListUtils; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; @@ -67,6 +69,7 @@ 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.index.semantic.ValueSetIndexes; import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; import javax.annotation.Nonnull; @@ -74,7 +77,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.NoSuchElementException; import java.util.SortedSet; @@ -195,6 +200,8 @@ public T as(Class clazz) return (T) (NullValueIndex) () -> nullIndex; } else if (clazz.equals(ValueIndexes.class)) { return (T) new DoubleValueIndexes(); + } else if (clazz.equals(ValueSetIndexes.class)) { + return (T) new DoubleValueSetIndexes(); } else if (clazz.equals(StringValueSetIndexes.class)) { return (T) new DoubleStringValueSetIndexes(); } else if (clazz.equals(NumericRangeIndexes.class)) { @@ -258,6 +265,73 @@ public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boo } } + private final class DoubleValueSetIndexes implements ValueSetIndexes + { + final FixedIndexed dictionary = doubleDictionarySupplier.get(); + + @Nullable + @Override + public BitmapColumnIndex forSortedValues(@Nonnull List sortedValues, TypeSignature matchValueType) + { + if (sortedValues.isEmpty()) { + return new AllFalseBitmapColumnIndex(bitmapFactory); + } + final boolean matchNull = sortedValues.get(0) == null; + final Supplier unknownsIndex = () -> { + if (!matchNull && dictionary.get(0) == null) { + return valueIndexes.get(0); + } + return null; + }; + + // values are doubles and ordered in double order + if (matchValueType.is(ValueType.DOUBLE)) { + final List tailSet; + final List baseSet = (List) sortedValues; + + if (sortedValues.size() >= ValueSetIndexes.SIZE_WORTH_CHECKING_MIN) { + final double minValueInColumn = dictionary.get(0) == null ? dictionary.get(1) : dictionary.get(0); + final int position = Collections.binarySearch( + sortedValues, + minValueInColumn, + matchValueType.getNullableStrategy() + ); + + tailSet = baseSet.subList(position >= 0 ? position : -(position + 1), baseSet.size()); + } else { + tailSet = baseSet; + } + if (tailSet.size() > ValueSetIndexes.SORTED_SCAN_RATIO_THRESHOLD * dictionary.size()) { + return ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorScan( + bitmapFactory, + ColumnType.DOUBLE.getNullableStrategy(), + tailSet, + dictionary, + valueIndexes, + unknownsIndex + ); + } + // fall through to sorted value iteration + return ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorBinarySearch( + bitmapFactory, + tailSet, + dictionary, + valueIndexes, + unknownsIndex + ); + } else { + // values in set are not sorted in double order, transform them on the fly and iterate them all + return ValueSetIndexes.buildBitmapColumnIndexFromIteratorBinarySearch( + bitmapFactory, + Iterables.transform(sortedValues, DimensionHandlerUtils::convertObjectToDouble), + dictionary, + valueIndexes, + unknownsIndex + ); + } + } + } + private class DoubleStringValueSetIndexes implements StringValueSetIndexes { @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 d2aa90e607e9..5f0f3a61efeb 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 @@ -21,6 +21,7 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import it.unimi.dsi.fastutil.ints.IntIntPair; import it.unimi.dsi.fastutil.ints.IntIterator; import it.unimi.dsi.fastutil.longs.LongArraySet; @@ -39,6 +40,7 @@ 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.DimensionHandlerUtils; import org.apache.druid.segment.IntListUtils; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnConfig; @@ -66,6 +68,7 @@ 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.index.semantic.ValueSetIndexes; import org.apache.druid.segment.serde.NestedCommonFormatColumnPartSerde; import javax.annotation.Nonnull; @@ -73,7 +76,9 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.NoSuchElementException; import java.util.SortedSet; @@ -195,6 +200,8 @@ public T as(Class clazz) return (T) (NullValueIndex) () -> nullIndex; } else if (clazz.equals(ValueIndexes.class)) { return (T) new LongValueIndexes(); + } else if (clazz.equals(ValueSetIndexes.class)) { + return (T) new LongValueSetIndexes(); } else if (clazz.equals(StringValueSetIndexes.class)) { return (T) new LongStringValueSetIndexes(); } else if (clazz.equals(NumericRangeIndexes.class)) { @@ -259,6 +266,73 @@ public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boo } } + private final class LongValueSetIndexes implements ValueSetIndexes + { + final FixedIndexed dictionary = longDictionarySupplier.get(); + + @Nullable + @Override + public BitmapColumnIndex forSortedValues(@Nonnull List sortedValues, TypeSignature matchValueType) + { + if (sortedValues.isEmpty()) { + return new AllFalseBitmapColumnIndex(bitmapFactory); + } + final boolean matchNull = sortedValues.get(0) == null; + final Supplier unknownsIndex = () -> { + if (!matchNull && dictionary.get(0) == null) { + return valueIndexes.get(0); + } + return null; + }; + if (matchValueType.is(ValueType.LONG)) { + final List tailSet; + final List baseSet = (List) sortedValues; + + if (sortedValues.size() >= ValueSetIndexes.SIZE_WORTH_CHECKING_MIN) { + final long minValueInColumn = dictionary.get(0) == null ? dictionary.get(1) : dictionary.get(0); + final int position = Collections.binarySearch( + sortedValues, + minValueInColumn, + matchValueType.getNullableStrategy() + ); + tailSet = baseSet.subList(position >= 0 ? position : -(position + 1), baseSet.size()); + } else { + tailSet = baseSet; + } + if (tailSet.size() > ValueSetIndexes.SORTED_SCAN_RATIO_THRESHOLD * dictionary.size()) { + return ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorScan( + bitmapFactory, + ColumnType.LONG.getNullableStrategy(), + tailSet, + dictionary, + valueIndexes, + unknownsIndex + ); + } + // fall through to sort value iteration + return ValueSetIndexes.buildBitmapColumnIndexFromSortedIteratorBinarySearch( + bitmapFactory, + tailSet, + dictionary, + valueIndexes, + unknownsIndex + ); + } else { + // values in set are not sorted in double order, transform them on the fly and iterate them all + return ValueSetIndexes.buildBitmapColumnIndexFromIteratorBinarySearch( + bitmapFactory, + Iterables.transform( + sortedValues, + DimensionHandlerUtils::convertObjectToLong + ), + dictionary, + valueIndexes, + unknownsIndex + ); + } + } + } + private class LongStringValueSetIndexes implements StringValueSetIndexes { final FixedIndexed dictionary = longDictionarySupplier.get(); @@ -324,7 +398,7 @@ public Iterable getBitmapIterable() if (value == null) { needNullCheck = true; } else { - Long theValue = GuavaUtils.tryParseLong(value); + Long theValue = DimensionHandlerUtils.convertObjectToLong(value); if (theValue != null) { longs.add(theValue.longValue()); if (NullHandling.replaceWithDefault() && theValue.equals(NullHandling.defaultLongValue())) { 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 23e86ce45a23..51e09434fb06 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 @@ -43,6 +43,7 @@ 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 org.apache.druid.segment.index.semantic.ValueSetIndexes; import javax.annotation.Nullable; import java.nio.ByteBuffer; @@ -100,7 +101,8 @@ public T as(Class clazz) } else if ( clazz.equals(StringValueSetIndexes.class) || clazz.equals(Utf8ValueSetIndexes.class) || - clazz.equals(ValueIndexes.class) + clazz.equals(ValueIndexes.class) || + clazz.equals(ValueSetIndexes.class) ) { return (T) new IndexedUtf8ValueIndexes<>( bitmapFactory, 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 3916954906c2..6cba418a3117 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 @@ -46,6 +46,10 @@ import java.io.Closeable; import java.util.List; +/** + * Classic {@link BoundFilter} test. Consider adding tests to {@link RangeFilterTests} in addition to, or instead of + * here. + */ @RunWith(Parameterized.class) public class BoundFilterTest extends BaseFilterTest { 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 deleted file mode 100644 index fd25166bcd17..000000000000 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java +++ /dev/null @@ -1,585 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.filter; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import nl.jqno.equalsverifier.EqualsVerifier; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.data.input.InputRow; -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.extraction.MapLookupExtractor; -import org.apache.druid.query.filter.DimFilter; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.filter.InDimFilter; -import org.apache.druid.query.filter.NotDimFilter; -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.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -import java.io.Closeable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -@RunWith(Parameterized.class) -public class InFilterTest extends BaseFilterTest -{ - private static final List ROWS = ImmutableList.of( - makeDefaultSchemaRow("a", "", ImmutableList.of("a", "b"), "2017-07-25", "", 0.0, 0.0f, 0L), - makeDefaultSchemaRow("b", "10", ImmutableList.of(), "2017-07-25", "a", 10.1, 10.1f, 100L), - makeDefaultSchemaRow("c", "2", ImmutableList.of(""), "2017-05-25", null, null, 5.5f, 40L), - makeDefaultSchemaRow("d", "1", ImmutableList.of("a"), "2020-01-25", "b", 120.0245, 110.0f, null), - makeDefaultSchemaRow("e", "def", ImmutableList.of("c"), null, "c", 60.0, null, 9001L), - makeDefaultSchemaRow("f", "abc", null, "2020-01-25", "a", 765.432, 123.45f, 12345L) - ); - - public InFilterTest( - 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(InFilterTest.class.getName()); - } - - @Test - public void testSingleValueStringColumnWithoutNulls() - { - assertFilterMatches( - toInFilter("dim0"), - ImmutableList.of() - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("dim0")), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - - assertFilterMatches( - toInFilter("dim0", null), - ImmutableList.of() - ); - - assertFilterMatches( - toInFilter("dim0", "", ""), - ImmutableList.of() - ); - - assertFilterMatches( - toInFilter("dim0", "a", "c"), - ImmutableList.of("a", "c") - ); - - assertFilterMatches( - toInFilter("dim0", "e", "x"), - ImmutableList.of("e") - ); - - assertFilterMatches( - NotDimFilter.of(toInFilter("dim0", "e", "x")), - ImmutableList.of("a", "b", "c", "d", "f") - ); - } - - @Test - public void testSingleValueStringColumnWithNulls() - { - assertFilterMatches( - toInFilter("dim1", null, ""), - ImmutableList.of("a") - ); - - assertFilterMatches( - toInFilter("dim1", ""), - ImmutableList.of("a") - ); - - assertFilterMatches( - toInFilter("dim1", "-1", "ab", "de"), - ImmutableList.of() - ); - - assertFilterMatches( - toInFilter("s0", "a", "b"), - ImmutableList.of("b", "d", "f") - ); - assertFilterMatches( - toInFilter("s0", "noexist"), - ImmutableList.of() - ); - - if (NullHandling.replaceWithDefault()) { - assertFilterMatches( - toInFilter("dim1", null, "10", "abc"), - ImmutableList.of("a", "b", "f") - ); - assertFilterMatches( - toInFilter("dim1", null, "10", "abc"), - ImmutableList.of("a", "b", "f") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("dim1", "-1", "ab", "de")), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("s0", "a", "b")), - ImmutableList.of("a", "c", "e") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("s0", "noexist")), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - } else { - assertFilterMatches( - toInFilter("dim1", null, "10", "abc"), - ImmutableList.of("b", "f") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("dim1", "-1", "ab", "de")), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("s0", "a", "b")), - ImmutableList.of("a", "e") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("s0", "noexist")), - ImmutableList.of("a", "b", "d", "e", "f") - ); - } - } - - @Test - public void testMultiValueStringColumn() - { - if (isAutoSchema()) { - return; - } - if (NullHandling.replaceWithDefault()) { - assertFilterMatches( - toInFilter("dim2", "b", "d"), - ImmutableList.of("a") - ); - assertFilterMatches( - toInFilter("dim2", null), - ImmutableList.of("b", "c", "f") - ); - assertFilterMatches( - toInFilter("dim2", null, "a"), - ImmutableList.of("a", "b", "c", "d", "f") - ); - assertFilterMatches( - toInFilter("dim2", null, "b"), - ImmutableList.of("a", "b", "c", "f") - ); - assertFilterMatches( - toInFilter("dim2", ""), - ImmutableList.of("b", "c", "f") - ); - } else { - assertFilterMatches( - toInFilter("dim2", null), - ImmutableList.of("b", "f") - ); - assertFilterMatches( - toInFilter("dim2", null, "a"), - ImmutableList.of("a", "b", "d", "f") - ); - assertFilterMatches( - toInFilter("dim2", null, "b"), - ImmutableList.of("a", "b", "f") - ); - assertFilterMatches( - toInFilter("dim2", ""), - ImmutableList.of("c") - ); - } - - assertFilterMatches( - toInFilter("dim2", "", (String) null), - ImmutableList.of("b", "c", "f") - ); - - assertFilterMatches( - toInFilter("dim2", "c"), - ImmutableList.of("e") - ); - - assertFilterMatches( - toInFilter("dim2", "d"), - ImmutableList.of() - ); - } - - @Test - public void testMissingColumn() - { - assertFilterMatches( - toInFilter("dim3", null, (String) null), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("dim3", null, (String) null)), - ImmutableList.of() - ); - - if (NullHandling.replaceWithDefault()) { - assertFilterMatches( - toInFilter("dim3", ""), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - } else { - assertFilterMatches( - toInFilter("dim3", ""), - ImmutableList.of() - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("dim3", "")), - ImmutableList.of() - ); - } - - assertFilterMatches( - toInFilter("dim3", null, "a"), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("dim3", null, "a")), - ImmutableList.of() - ); - - assertFilterMatches( - toInFilter("dim3", "a"), - ImmutableList.of() - ); - assertFilterMatches( - NotDimFilter.of(toInFilter("dim3", "a")), - NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - - assertFilterMatches( - toInFilter("dim3", "b"), - ImmutableList.of() - ); - - assertFilterMatches( - toInFilter("dim3", "c"), - ImmutableList.of() - ); - } - - @Test - public void testMatchWithExtractionFn() - { - String extractionJsFn = "function(str) { return 'super-' + str; }"; - ExtractionFn superFn = new JavaScriptExtractionFn(extractionJsFn, false, JavaScriptConfig.getEnabledInstance()); - - String nullJsFn = "function(str) { if (str === null) { return 'YES'; } else { return 'NO';} }"; - ExtractionFn yesNullFn = new JavaScriptExtractionFn(nullJsFn, false, JavaScriptConfig.getEnabledInstance()); - - if (NullHandling.replaceWithDefault()) { - assertFilterMatchesSkipArrays( - toInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b"), - ImmutableList.of("a", "b", "c", "d", "f") - ); - assertFilterMatchesSkipArrays( - NotDimFilter.of(toInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b")), - ImmutableList.of("e") - ); - assertFilterMatchesSkipArrays( - toInFilterWithFn("dim2", yesNullFn, "YES"), - ImmutableList.of("b", "c", "f") - ); - assertFilterMatchesSkipArrays( - NotDimFilter.of(toInFilterWithFn("dim2", yesNullFn, "YES")), - ImmutableList.of("a", "d", "e") - ); - assertFilterMatches( - toInFilterWithFn("dim1", superFn, "super-null", "super-10", "super-def"), - ImmutableList.of("a", "b", "e") - ); - assertFilterMatches( - toInFilterWithFn("dim1", yesNullFn, "NO"), - ImmutableList.of("b", "c", "d", "e", "f") - ); - } else { - assertFilterMatchesSkipArrays( - toInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b"), - ImmutableList.of("a", "b", "d", "f") - ); - assertFilterMatchesSkipArrays( - NotDimFilter.of(toInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b")), - ImmutableList.of("c", "e") - ); - assertFilterMatchesSkipArrays( - toInFilterWithFn("dim2", yesNullFn, "YES"), - ImmutableList.of("b", "f") - ); - assertFilterMatchesSkipArrays( - NotDimFilter.of(toInFilterWithFn("dim2", yesNullFn, "YES")), - ImmutableList.of("a", "c", "d", "e") - ); - assertFilterMatches( - toInFilterWithFn("dim1", superFn, "super-null", "super-10", "super-def"), - ImmutableList.of("b", "e") - ); - - assertFilterMatches( - toInFilterWithFn("dim1", yesNullFn, "NO"), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - } - - - assertFilterMatches( - toInFilterWithFn("dim3", yesNullFn, "NO"), - ImmutableList.of() - ); - assertFilterMatches( - NotDimFilter.of(toInFilterWithFn("dim3", yesNullFn, "NO")), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - assertFilterMatches( - toInFilterWithFn("dim3", yesNullFn, "YES"), - ImmutableList.of("a", "b", "c", "d", "e", "f") - ); - - } - - @Test - public void testMatchWithLookupExtractionFn() - { - final Map stringMap = ImmutableMap.of( - "a", "HELLO", - "10", "HELLO", - "def", "HELLO", - "c", "BYE" - ); - LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false); - LookupExtractionFn lookupFn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true); - - assertFilterMatches(toInFilterWithFn("dim0", lookupFn, null, "HELLO"), ImmutableList.of("a")); - assertFilterMatches(toInFilterWithFn("dim0", lookupFn, "HELLO", "BYE"), ImmutableList.of("a", "c")); - 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()); - - if (optimize) { - // Arrays don't cause errors when the extractionFn is optimized, because the "IN" filter vanishes completely. - assertFilterMatches(toInFilterWithFn("dim2", lookupFn, "a"), ImmutableList.of()); - } else { - 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") - ); - - final Map stringMap2 = ImmutableMap.of( - "a", "e" - ); - LookupExtractor mapExtractor2 = new MapLookupExtractor(stringMap2, false); - LookupExtractionFn lookupFn2 = new LookupExtractionFn(mapExtractor2, true, null, false, true); - - assertFilterMatches(toInFilterWithFn("dim0", lookupFn2, null, "e"), ImmutableList.of("a", "e")); - assertFilterMatches(toInFilterWithFn("dim0", lookupFn2, "a"), ImmutableList.of()); - - final Map stringMap3 = ImmutableMap.of( - "c", "500", - "100", "e" - ); - LookupExtractor mapExtractor3 = new MapLookupExtractor(stringMap3, false); - LookupExtractionFn lookupFn3 = new LookupExtractionFn(mapExtractor3, false, null, false, true); - - assertFilterMatches(toInFilterWithFn("dim0", lookupFn3, null, "c"), ImmutableList.of("a", "b", "d", "e", "f")); - assertFilterMatches(toInFilterWithFn("dim0", lookupFn3, "e"), ImmutableList.of()); - - } - - @Test - public void testNumericColumnNullsAndDefaults() - { - if (canTestNumericNullsAsDefaultValues) { - assertFilterMatches(new InDimFilter("f0", Sets.newHashSet("0"), null), ImmutableList.of("a", "e")); - assertFilterMatches(new InDimFilter("d0", Sets.newHashSet("0"), null), ImmutableList.of("a", "c")); - assertFilterMatches(new InDimFilter("l0", Sets.newHashSet("0"), null), ImmutableList.of("a", "d")); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("f0", Sets.newHashSet("0"), null)), - ImmutableList.of("b", "c", "d", "f") - ); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("d0", Sets.newHashSet("0"), null)), - ImmutableList.of("b", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("l0", Sets.newHashSet("0"), null)), - ImmutableList.of("b", "c", "e", "f") - ); - assertFilterMatches(new InDimFilter("f0", Collections.singleton(null), null), ImmutableList.of()); - assertFilterMatches(new InDimFilter("d0", Collections.singleton(null), null), ImmutableList.of()); - assertFilterMatches(new InDimFilter("l0", Collections.singleton(null), null), ImmutableList.of()); - - assertFilterMatches(new InDimFilter("f0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a", "e")); - assertFilterMatches(new InDimFilter("d0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a", "c")); - assertFilterMatches(new InDimFilter("l0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a", "d")); - assertFilterMatches(new InDimFilter("f0", Sets.newHashSet(null, "999"), null), ImmutableList.of()); - assertFilterMatches(new InDimFilter("d0", Sets.newHashSet(null, "999"), null), ImmutableList.of()); - assertFilterMatches(new InDimFilter("l0", Sets.newHashSet(null, "999"), null), ImmutableList.of()); - } else { - assertFilterMatches(new InDimFilter("f0", Sets.newHashSet("0"), null), ImmutableList.of("a")); - assertFilterMatches(new InDimFilter("d0", Sets.newHashSet("0"), null), ImmutableList.of("a")); - assertFilterMatches(new InDimFilter("l0", Sets.newHashSet("0"), null), ImmutableList.of("a")); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("f0", Sets.newHashSet("0"), null)), - NullHandling.sqlCompatible() - ? ImmutableList.of("b", "c", "d", "f") - : ImmutableList.of("b", "c", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("d0", Sets.newHashSet("0"), null)), - NullHandling.sqlCompatible() - ? ImmutableList.of("b", "d", "e", "f") - : ImmutableList.of("b", "c", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("l0", Sets.newHashSet("0"), null)), - NullHandling.sqlCompatible() - ? ImmutableList.of("b", "c", "e", "f") - : ImmutableList.of("b", "c", "d", "e", "f") - ); - assertFilterMatches(new InDimFilter("f0", Collections.singleton(null), null), ImmutableList.of("e")); - assertFilterMatches(new InDimFilter("d0", Collections.singleton(null), null), ImmutableList.of("c")); - assertFilterMatches(new InDimFilter("l0", Collections.singleton(null), null), ImmutableList.of("d")); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("f0", Collections.singleton(null), null)), - ImmutableList.of("a", "b", "c", "d", "f") - ); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("d0", Collections.singleton(null), null)), - ImmutableList.of("a", "b", "d", "e", "f") - ); - assertFilterMatches( - NotDimFilter.of(new InDimFilter("l0", Collections.singleton(null), null)), - ImmutableList.of("a", "b", "c", "e", "f") - ); - - assertFilterMatches(new InDimFilter("f0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a")); - assertFilterMatches(new InDimFilter("d0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a")); - assertFilterMatches(new InDimFilter("l0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a")); - assertFilterMatches(new InDimFilter("f0", Sets.newHashSet(null, "999"), null), ImmutableList.of("e")); - assertFilterMatches(new InDimFilter("d0", Sets.newHashSet(null, "999"), null), ImmutableList.of("c")); - assertFilterMatches(new InDimFilter("l0", Sets.newHashSet(null, "999"), null), ImmutableList.of("d")); - } - } - - @Test - public void testRequiredColumnRewrite() - { - InDimFilter filter = (InDimFilter) toInFilter("dim0", "a", "c").toFilter(); - InDimFilter filter2 = (InDimFilter) toInFilter("dim1", "a", "c").toFilter(); - - 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 test_equals() - { - EqualsVerifier.forClass(InDimFilter.class) - .usingGetClass() - .withNonnullFields("dimension", "values") - .withIgnoredFields( - "cacheKeySupplier", - "predicateFactory", - "optimizedFilterIncludeUnknown", - "optimizedFilterNoIncludeUnknown", - "valuesUtf8" - ) - .verify(); - } - - @Test - public void test_equals_forInFilterDruidPredicateFactory() - { - EqualsVerifier.forClass(InDimFilter.InFilterDruidPredicateFactory.class) - .usingGetClass() - .withNonnullFields("values") - .withIgnoredFields( - "longPredicateSupplier", - "floatPredicateSupplier", - "doublePredicateSupplier", - "stringPredicateSupplier" - ) - .verify(); - } - - private DimFilter toInFilter(String dim) - { - List emptyList = new ArrayList<>(); - return new InDimFilter(dim, emptyList, null); - } - - private DimFilter toInFilter(String dim, String value, String... values) - { - return new InDimFilter(dim, Lists.asList(value, values), null); - } - - private DimFilter toInFilterWithFn(String dim, ExtractionFn fn, String value, String... values) - { - return new InDimFilter(dim, Lists.asList(value, values), fn); - } -} diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java new file mode 100644 index 000000000000..46f120922392 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTests.java @@ -0,0 +1,978 @@ +/* + * 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.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.RangeSet; +import com.google.common.collect.Sets; +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.jackson.DefaultObjectMapper; +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.extraction.MapLookupExtractor; +import org.apache.druid.query.filter.DimFilter; +import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.TypedInFilter; +import org.apache.druid.query.lookup.LookupExtractionFn; +import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import javax.annotation.Nullable; +import java.io.Closeable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + + +public class InFilterTests +{ + @RunWith(Parameterized.class) + public static class InFilterTest extends BaseFilterTest + { + private static final List ROWS = ImmutableList.of( + makeDefaultSchemaRow("a", "", ImmutableList.of("a", "b"), "2017-07-25", "", 0.0, 0.0f, 0L), + makeDefaultSchemaRow("b", "10", ImmutableList.of(), "2017-07-25", "a", 10.1, 10.1f, 100L), + makeDefaultSchemaRow("c", "2", ImmutableList.of(""), "2017-05-25", null, null, 5.5f, 40L), + makeDefaultSchemaRow("d", "1", ImmutableList.of("a"), "2020-01-25", "b", 120.0245, 110.0f, null), + makeDefaultSchemaRow("e", "def", ImmutableList.of("c"), null, "c", 60.0, null, 9001L), + makeDefaultSchemaRow("f", "abc", null, "2020-01-25", "a", 765.432, 123.45f, 12345L) + ); + + private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + public InFilterTest( + 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(InFilterTest.class.getName()); + } + + @Test + public void testSingleValueStringColumnWithoutNulls() + { + assertFilterMatches( + inFilter("dim0", ColumnType.STRING, Collections.emptyList()), + ImmutableList.of() + ); + assertFilterMatches( + NotDimFilter.of(inFilter("dim0", ColumnType.STRING, Collections.emptyList())), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + + assertFilterMatches( + inFilter("dim0", ColumnType.STRING, Collections.singletonList(null)), + ImmutableList.of() + ); + + assertFilterMatches( + inFilter("dim0", ColumnType.STRING, Arrays.asList("", "")), + ImmutableList.of() + ); + + assertFilterMatches( + inFilter("dim0", ColumnType.STRING, Arrays.asList("a", "c")), + ImmutableList.of("a", "c") + ); + + assertFilterMatches( + inFilter("dim0", ColumnType.STRING, Arrays.asList("e", "x")), + ImmutableList.of("e") + ); + + assertFilterMatches( + NotDimFilter.of(inFilter("dim0", ColumnType.STRING, Arrays.asList("e", "x"))), + ImmutableList.of("a", "b", "c", "d", "f") + ); + } + @Test + public void testSingleValueStringColumnWithNulls() + { + assertFilterMatches( + inFilter("dim1", ColumnType.STRING, Arrays.asList(null, "")), + ImmutableList.of("a") + ); + + assertFilterMatches( + inFilter("dim1", ColumnType.STRING, Collections.singletonList("")), + ImmutableList.of("a") + ); + + assertFilterMatches( + inFilter("dim1", ColumnType.STRING, Arrays.asList("-1", "ab", "de")), + ImmutableList.of() + ); + + assertFilterMatches( + inFilter("s0", ColumnType.STRING, Arrays.asList("a", "b")), + ImmutableList.of("b", "d", "f") + ); + + assertFilterMatches( + inFilter("s0", ColumnType.STRING, Collections.singletonList("noexist")), + ImmutableList.of() + ); + + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + inFilter("dim1", ColumnType.STRING, Arrays.asList(null, "10", "abc")), + ImmutableList.of("b", "f") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("dim1", ColumnType.STRING, Arrays.asList("-1", "ab", "de"))), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("s0", ColumnType.STRING, Arrays.asList("a", "b"))), + ImmutableList.of("a", "e") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("s0", ColumnType.STRING, Collections.singletonList("noexist"))), + ImmutableList.of("a", "b", "d", "e", "f") + ); + } else { + // typed in filter doesn't support default value mode, so use classic filter only + assertLegacyFilterMatches( + legacyInFilter("dim1", null, "10", "abc"), + ImmutableList.of("a", "b", "f") + ); + assertLegacyFilterMatches( + legacyInFilter("dim1", null, "10", "abc"), + ImmutableList.of("a", "b", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(legacyInFilter("dim1", "-1", "ab", "de")), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(legacyInFilter("s0", "a", "b")), + ImmutableList.of("a", "c", "e") + ); + assertLegacyFilterMatches( + NotDimFilter.of(legacyInFilter("s0", "noexist")), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + } + } + + @Test + public void testMultiValueStringColumn() + { + Assume.assumeFalse(isAutoSchema()); + + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + inFilter("dim2", ColumnType.STRING, Collections.singletonList(null)), + ImmutableList.of("b", "f") + ); + assertFilterMatches( + inFilter("dim2", ColumnType.STRING, Arrays.asList(null, "a")), + ImmutableList.of("a", "b", "d", "f") + ); + assertFilterMatches( + inFilter("dim2", ColumnType.STRING, Arrays.asList(null, "b")), + ImmutableList.of("a", "b", "f") + ); + assertFilterMatches( + inFilter("dim2", ColumnType.STRING, Collections.singletonList("")), + ImmutableList.of("c") + ); + } else { + assertLegacyFilterMatches( + legacyInFilter("dim2", "b", "d"), + ImmutableList.of("a") + ); + assertLegacyFilterMatches( + legacyInFilter("dim2", null), + ImmutableList.of("b", "c", "f") + ); + assertLegacyFilterMatches( + legacyInFilter("dim2", null, "a"), + ImmutableList.of("a", "b", "c", "d", "f") + ); + assertLegacyFilterMatches( + legacyInFilter("dim2", null, "b"), + ImmutableList.of("a", "b", "c", "f") + ); + assertLegacyFilterMatches( + legacyInFilter("dim2", ""), + ImmutableList.of("b", "c", "f") + ); + } + + assertFilterMatches( + inFilter("dim2", ColumnType.STRING, Arrays.asList("", null)), + ImmutableList.of("b", "c", "f") + ); + + assertFilterMatches( + inFilter("dim2", ColumnType.STRING, Collections.singletonList("c")), + ImmutableList.of("e") + ); + + assertFilterMatches( + inFilter("dim2", ColumnType.STRING, Collections.singletonList("d")), + ImmutableList.of() + ); + } + + @Test + public void testMissingColumn() + { + assertFilterMatches( + inFilter("dim3", ColumnType.STRING, Arrays.asList(null, null)), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("dim3", ColumnType.STRING, Arrays.asList(null, null))), + ImmutableList.of() + ); + + assertFilterMatches( + inFilter("dim3", ColumnType.STRING, Arrays.asList(null, "a")), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + assertFilterMatches( + inFilter("dim3", ColumnType.STRING, Collections.singletonList("a")), + ImmutableList.of() + ); + assertFilterMatches( + inFilter("dim3", ColumnType.STRING, Collections.singletonList("b")), + ImmutableList.of() + ); + assertFilterMatches( + inFilter("dim3", ColumnType.STRING, Collections.singletonList("c")), + ImmutableList.of() + ); + + + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + inFilter("dim3", ColumnType.STRING, Collections.singletonList("")), + ImmutableList.of() + ); + assertFilterMatches( + NotDimFilter.of(inFilter("dim3", ColumnType.STRING, Collections.singletonList(""))), + ImmutableList.of() + ); + assertFilterMatches( + NotDimFilter.of(inFilter("dim3", ColumnType.STRING, Collections.singletonList("a"))), + ImmutableList.of() + ); + assertFilterMatches( + NotDimFilter.of(inFilter("dim3", ColumnType.STRING, Arrays.asList(null, "a"))), + ImmutableList.of() + ); + } else { + assertLegacyFilterMatches( + legacyInFilter("dim3", ""), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(legacyInFilter("dim3", "")), + ImmutableList.of() + ); + assertLegacyFilterMatches( + NotDimFilter.of(legacyInFilter("dim3", "a")), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(legacyInFilter("dim3", null, "a")), + ImmutableList.of() + ); + } + } + + @Test + public void testNumeric() + { + Assume.assumeTrue(NullHandling.sqlCompatible()); + assertFilterMatches( + inFilter("f0", ColumnType.FLOAT, Collections.singletonList(0f)), + ImmutableList.of("a") + ); + assertFilterMatches( + inFilter("d0", ColumnType.DOUBLE, Collections.singletonList(0.0)), + ImmutableList.of("a") + ); + assertFilterMatches(inFilter("l0", ColumnType.LONG, Collections.singletonList(0L)), ImmutableList.of("a")); + assertFilterMatches( + NotDimFilter.of(inFilter("f0", ColumnType.FLOAT, Collections.singletonList(0f))), + ImmutableList.of("b", "c", "d", "f") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("d0", ColumnType.DOUBLE, Collections.singletonList(0.0))), + ImmutableList.of("b", "d", "e", "f") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("l0", ColumnType.LONG, Collections.singletonList(0L))), + ImmutableList.of("b", "c", "e", "f") + ); + assertFilterMatches(inFilter("f0", ColumnType.FLOAT, Collections.singletonList(null)), ImmutableList.of("e")); + assertFilterMatches(inFilter("d0", ColumnType.DOUBLE, Collections.singletonList(null)), ImmutableList.of("c")); + assertFilterMatches(inFilter("l0", ColumnType.LONG, Collections.singletonList(null)), ImmutableList.of("d")); + assertFilterMatches( + NotDimFilter.of(inFilter("f0", ColumnType.FLOAT, Collections.singletonList(null))), + ImmutableList.of("a", "b", "c", "d", "f") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("d0", ColumnType.DOUBLE, Collections.singletonList(null))), + ImmutableList.of("a", "b", "d", "e", "f") + ); + assertFilterMatches( + NotDimFilter.of(inFilter("l0", ColumnType.LONG, Collections.singletonList(null))), + ImmutableList.of("a", "b", "c", "e", "f") + ); + + assertFilterMatches(inFilter("f0", ColumnType.FLOAT, Arrays.asList("0", "999")), ImmutableList.of("a")); + assertFilterMatches(inFilter("d0", ColumnType.DOUBLE, Arrays.asList("0", "999")), ImmutableList.of("a")); + assertFilterMatches(inFilter("l0", ColumnType.LONG, Arrays.asList("0", "999")), ImmutableList.of("a")); + assertFilterMatches(inFilter("f0", ColumnType.FLOAT, Arrays.asList(null, "999")), ImmutableList.of("e")); + assertFilterMatches(inFilter("d0", ColumnType.DOUBLE, Arrays.asList(null, "999")), ImmutableList.of("c")); + assertFilterMatches(inFilter("l0", ColumnType.LONG, Arrays.asList(null, "999")), ImmutableList.of("d")); + + assertFilterMatches( + inFilter("l0", ColumnType.LONG, Arrays.asList(100L, 9001L)), + ImmutableList.of("b", "e") + ); + assertFilterMatches( + inFilter("l0", ColumnType.FLOAT, Arrays.asList(100.0f, 110.0f)), + ImmutableList.of("b") + ); + assertFilterMatches( + inFilter("l0", ColumnType.DOUBLE, Arrays.asList(100.0, 110.0)), + ImmutableList.of("b") + ); + + assertFilterMatches( + inFilter("d0", ColumnType.DOUBLE, Arrays.asList(10.1, 120.0245)), + ImmutableList.of("b", "d") + ); + + // auto schema doesn't have float columns, so these get kind of funny + Assume.assumeFalse(isAutoSchema()); + assertFilterMatches( + inFilter("f0", ColumnType.FLOAT, Arrays.asList(10.1f, 110.0f)), + ImmutableList.of("b", "d") + ); + assertFilterMatches( + inFilter("f0", ColumnType.DOUBLE, Arrays.asList(10.1, 110.0)), + ImmutableList.of("b", "d") + ); + } + + @Test + public void testLegacyNumericDefaults() + { + if (canTestNumericNullsAsDefaultValues) { + assertLegacyFilterMatches(new InDimFilter("f0", Sets.newHashSet("0"), null), ImmutableList.of("a", "e")); + assertLegacyFilterMatches(new InDimFilter("d0", Sets.newHashSet("0"), null), ImmutableList.of("a", "c")); + assertLegacyFilterMatches(new InDimFilter("l0", Sets.newHashSet("0"), null), ImmutableList.of("a", "d")); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("f0", Sets.newHashSet("0"), null)), + ImmutableList.of("b", "c", "d", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("d0", Sets.newHashSet("0"), null)), + ImmutableList.of("b", "d", "e", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("l0", Sets.newHashSet("0"), null)), + ImmutableList.of("b", "c", "e", "f") + ); + assertLegacyFilterMatches(new InDimFilter("f0", Collections.singleton(null), null), ImmutableList.of()); + assertLegacyFilterMatches(new InDimFilter("d0", Collections.singleton(null), null), ImmutableList.of()); + assertLegacyFilterMatches(new InDimFilter("l0", Collections.singleton(null), null), ImmutableList.of()); + + assertLegacyFilterMatches(new InDimFilter("f0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a", "e")); + assertLegacyFilterMatches(new InDimFilter("d0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a", "c")); + assertLegacyFilterMatches(new InDimFilter("l0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a", "d")); + assertLegacyFilterMatches(new InDimFilter("f0", Sets.newHashSet(null, "999"), null), ImmutableList.of()); + assertLegacyFilterMatches(new InDimFilter("d0", Sets.newHashSet(null, "999"), null), ImmutableList.of()); + assertLegacyFilterMatches(new InDimFilter("l0", Sets.newHashSet(null, "999"), null), ImmutableList.of()); + } else { + assertLegacyFilterMatches(new InDimFilter("f0", Sets.newHashSet("0"), null), ImmutableList.of("a")); + assertLegacyFilterMatches(new InDimFilter("d0", Sets.newHashSet("0"), null), ImmutableList.of("a")); + assertLegacyFilterMatches(new InDimFilter("l0", Sets.newHashSet("0"), null), ImmutableList.of("a")); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("f0", Sets.newHashSet("0"), null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("b", "c", "d", "f") + : ImmutableList.of("b", "c", "d", "e", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("d0", Sets.newHashSet("0"), null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("b", "d", "e", "f") + : ImmutableList.of("b", "c", "d", "e", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("l0", Sets.newHashSet("0"), null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("b", "c", "e", "f") + : ImmutableList.of("b", "c", "d", "e", "f") + ); + assertLegacyFilterMatches(new InDimFilter("f0", Collections.singleton(null), null), ImmutableList.of("e")); + assertLegacyFilterMatches(new InDimFilter("d0", Collections.singleton(null), null), ImmutableList.of("c")); + assertLegacyFilterMatches(new InDimFilter("l0", Collections.singleton(null), null), ImmutableList.of("d")); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("f0", Collections.singleton(null), null)), + ImmutableList.of("a", "b", "c", "d", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("d0", Collections.singleton(null), null)), + ImmutableList.of("a", "b", "d", "e", "f") + ); + assertLegacyFilterMatches( + NotDimFilter.of(new InDimFilter("l0", Collections.singleton(null), null)), + ImmutableList.of("a", "b", "c", "e", "f") + ); + + assertLegacyFilterMatches(new InDimFilter("f0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a")); + assertLegacyFilterMatches(new InDimFilter("d0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a")); + assertLegacyFilterMatches(new InDimFilter("l0", Sets.newHashSet("0", "999"), null), ImmutableList.of("a")); + assertLegacyFilterMatches(new InDimFilter("f0", Sets.newHashSet(null, "999"), null), ImmutableList.of("e")); + assertLegacyFilterMatches(new InDimFilter("d0", Sets.newHashSet(null, "999"), null), ImmutableList.of("c")); + assertLegacyFilterMatches(new InDimFilter("l0", Sets.newHashSet(null, "999"), null), ImmutableList.of("d")); + } + } + @Test + public void testLegacyMatchWithExtractionFn() + { + String extractionJsFn = "function(str) { return 'super-' + str; }"; + ExtractionFn superFn = new JavaScriptExtractionFn(extractionJsFn, false, JavaScriptConfig.getEnabledInstance()); + + String nullJsFn = "function(str) { if (str === null) { return 'YES'; } else { return 'NO';} }"; + ExtractionFn yesNullFn = new JavaScriptExtractionFn(nullJsFn, false, JavaScriptConfig.getEnabledInstance()); + + if (NullHandling.replaceWithDefault()) { + assertFilterMatchesSkipArrays( + legacyInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b"), + ImmutableList.of("a", "b", "c", "d", "f") + ); + assertFilterMatchesSkipArrays( + NotDimFilter.of(legacyInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b")), + ImmutableList.of("e") + ); + assertFilterMatchesSkipArrays( + legacyInFilterWithFn("dim2", yesNullFn, "YES"), + ImmutableList.of("b", "c", "f") + ); + assertFilterMatchesSkipArrays( + NotDimFilter.of(legacyInFilterWithFn("dim2", yesNullFn, "YES")), + ImmutableList.of("a", "d", "e") + ); + assertLegacyFilterMatches( + legacyInFilterWithFn("dim1", superFn, "super-null", "super-10", "super-def"), + ImmutableList.of("a", "b", "e") + ); + assertLegacyFilterMatches( + legacyInFilterWithFn("dim1", yesNullFn, "NO"), + ImmutableList.of("b", "c", "d", "e", "f") + ); + } else { + assertFilterMatchesSkipArrays( + legacyInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b"), + ImmutableList.of("a", "b", "d", "f") + ); + assertFilterMatchesSkipArrays( + NotDimFilter.of(legacyInFilterWithFn("dim2", superFn, "super-null", "super-a", "super-b")), + ImmutableList.of("c", "e") + ); + assertFilterMatchesSkipArrays( + legacyInFilterWithFn("dim2", yesNullFn, "YES"), + ImmutableList.of("b", "f") + ); + assertFilterMatchesSkipArrays( + NotDimFilter.of(legacyInFilterWithFn("dim2", yesNullFn, "YES")), + ImmutableList.of("a", "c", "d", "e") + ); + assertLegacyFilterMatches( + legacyInFilterWithFn("dim1", superFn, "super-null", "super-10", "super-def"), + ImmutableList.of("b", "e") + ); + + assertLegacyFilterMatches( + legacyInFilterWithFn("dim1", yesNullFn, "NO"), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + } + + assertLegacyFilterMatches( + legacyInFilterWithFn("dim3", yesNullFn, "NO"), + ImmutableList.of() + ); + assertLegacyFilterMatches( + NotDimFilter.of(legacyInFilterWithFn("dim3", yesNullFn, "NO")), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + assertLegacyFilterMatches( + legacyInFilterWithFn("dim3", yesNullFn, "YES"), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + + } + + @Test + public void testLegacyMatchWithLookupExtractionFn() + { + final Map stringMap = ImmutableMap.of( + "a", "HELLO", + "10", "HELLO", + "def", "HELLO", + "c", "BYE" + ); + LookupExtractor mapExtractor = new MapLookupExtractor(stringMap, false); + LookupExtractionFn lookupFn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true); + + assertLegacyFilterMatches(legacyInFilterWithFn("dim0", lookupFn, null, "HELLO"), ImmutableList.of("a")); + assertLegacyFilterMatches(legacyInFilterWithFn("dim0", lookupFn, "HELLO", "BYE"), ImmutableList.of("a", "c")); + assertLegacyFilterMatches(legacyInFilterWithFn("dim0", lookupFn, "UNKNOWN"), ImmutableList.of("b", "d", "e", "f")); + assertLegacyFilterMatches(legacyInFilterWithFn("dim1", lookupFn, "HELLO"), ImmutableList.of("b", "e")); + assertLegacyFilterMatches(legacyInFilterWithFn("dim1", lookupFn, "N/A"), ImmutableList.of()); + + if (optimize) { + // Arrays don't cause errors when the extractionFn is optimized, because the "IN" filter vanishes completely. + assertLegacyFilterMatches(legacyInFilterWithFn("dim2", lookupFn, "a"), ImmutableList.of()); + } else { + assertFilterMatchesSkipArrays(legacyInFilterWithFn("dim2", lookupFn, "a"), ImmutableList.of()); + } + + assertFilterMatchesSkipArrays(legacyInFilterWithFn("dim2", lookupFn, "HELLO"), ImmutableList.of("a", "d")); + assertFilterMatchesSkipArrays( + legacyInFilterWithFn("dim2", lookupFn, "HELLO", "BYE", "UNKNOWN"), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); + + final Map stringMap2 = ImmutableMap.of( + "a", "e" + ); + LookupExtractor mapExtractor2 = new MapLookupExtractor(stringMap2, false); + LookupExtractionFn lookupFn2 = new LookupExtractionFn(mapExtractor2, true, null, false, true); + + assertLegacyFilterMatches(legacyInFilterWithFn("dim0", lookupFn2, null, "e"), ImmutableList.of("a", "e")); + assertLegacyFilterMatches(legacyInFilterWithFn("dim0", lookupFn2, "a"), ImmutableList.of()); + + final Map stringMap3 = ImmutableMap.of( + "c", "500", + "100", "e" + ); + LookupExtractor mapExtractor3 = new MapLookupExtractor(stringMap3, false); + LookupExtractionFn lookupFn3 = new LookupExtractionFn(mapExtractor3, false, null, false, true); + + assertLegacyFilterMatches(legacyInFilterWithFn("dim0", lookupFn3, null, "c"), ImmutableList.of("a", "b", "d", "e", "f")); + assertLegacyFilterMatches(legacyInFilterWithFn("dim0", lookupFn3, "e"), ImmutableList.of()); + } + + @Override + protected void assertFilterMatches(DimFilter filter, List expectedRows) + { + assertTypedFilterMatches(filter, expectedRows); + assertLegacyFilterMatches(filter, expectedRows); + } + + private void assertTypedFilterMatches(DimFilter filter, List expectedRows) + { + // this filter only tests in sql compatible mode + if (NullHandling.sqlCompatible()) { + super.assertFilterMatches(filter, expectedRows); + try { + // make sure round trip json serde is cool + super.assertFilterMatches( + jsonMapper.readValue(jsonMapper.writeValueAsString(filter), DimFilter.class), + expectedRows + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } else { + Throwable t = Assert.assertThrows( + DruidException.class, + () -> super.assertFilterMatches(filter, expectedRows) + ); + Assert.assertEquals("Invalid IN filter, typed in filter only supports SQL compatible null handling mode, set druid.generic.useDefaultValue=false to use this filter", t.getMessage()); + } + } + + private void assertLegacyFilterMatches(DimFilter filter, List expectedRows) + { + DimFilter newFilter = rewriteToLegacyFilter(filter); + if (newFilter != null) { + super.assertFilterMatches(newFilter, expectedRows); + } + } + + @Nullable + private DimFilter rewriteToLegacyFilter(DimFilter filter) + { + if (filter instanceof InDimFilter) { + return filter; + } else if (filter instanceof TypedInFilter) { + TypedInFilter theFilter = (TypedInFilter) filter; + return new InDimFilter( + theFilter.getColumn(), + InDimFilter.ValuesSet.copyOf( + theFilter.getSortedValues() + .stream() + .map(DimensionHandlerUtils::convertObjectToString) + .collect(Collectors.toList()) + ), + null + ); + } else if (filter instanceof NotDimFilter) { + DimFilter rewrite = rewriteToLegacyFilter(((NotDimFilter) filter).getField()); + if (rewrite != null) { + return NotDimFilter.of(rewrite); + } + } + return null; + } + } + + public static class TypedInFilterFilterNonParameterizedTests extends InitializedNullHandlingTest + { + @Test + public void testSerde() throws JsonProcessingException + { + Assume.assumeTrue(NullHandling.sqlCompatible()); + ObjectMapper mapper = new DefaultObjectMapper(); + TypedInFilter filter = inFilter("column", ColumnType.STRING, Arrays.asList("a", "b", "c")); + String s = mapper.writeValueAsString(filter); + Assert.assertEquals(filter, mapper.readValue(s, TypedInFilter.class)); + + filter = inFilter("column", ColumnType.STRING, Arrays.asList("a", "b", "b", null, "c")); + s = mapper.writeValueAsString(filter); + TypedInFilter deserialized = mapper.readValue(s, TypedInFilter.class); + Assert.assertEquals(Arrays.asList(null, "a", "b", "c"), deserialized.getSortedValues()); + Assert.assertEquals(filter, deserialized); + + filter = inFilter("column", ColumnType.LONG, Arrays.asList(1L, 2L, 2L, null, 3L)); + s = mapper.writeValueAsString(filter); + Assert.assertEquals(filter, mapper.readValue(s, TypedInFilter.class)); + + filter = inFilter("column", ColumnType.DOUBLE, Arrays.asList(1.1, 2.2, 2.3, null, 3.3)); + s = mapper.writeValueAsString(filter); + Assert.assertEquals(filter, mapper.readValue(s, TypedInFilter.class)); + + filter = inFilter("column", ColumnType.FLOAT, Arrays.asList(1.1f, 2.2f, 2.2f, null, 3.3f)); + s = mapper.writeValueAsString(filter); + Assert.assertEquals(filter, mapper.readValue(s, TypedInFilter.class)); + + filter = inFilter("column", ColumnType.FLOAT, Arrays.asList(1.1, 2.2, 2.3, null, 3.3)); + s = mapper.writeValueAsString(filter); + Assert.assertEquals(filter, mapper.readValue(s, TypedInFilter.class)); + } + + @Test + public void testGetCacheKey() + { + Assume.assumeTrue(NullHandling.sqlCompatible()); + TypedInFilter filterUnsorted = inFilter("column", ColumnType.STRING, Arrays.asList("a", "b", null, "c")); + TypedInFilter filterDifferent = inFilter("column", ColumnType.STRING, Arrays.asList("a", "c", "b")); + TypedInFilter filterPresorted = new TypedInFilter( + "column", + ColumnType.STRING, + null, + Arrays.asList(null, "a", "b", "c"), + null + ); + + Assert.assertEquals(filterPresorted, filterUnsorted); + Assert.assertNotEquals(filterDifferent, filterPresorted); + Assert.assertArrayEquals(filterPresorted.getCacheKey(), filterUnsorted.getCacheKey()); + Assert.assertFalse(Arrays.equals(filterDifferent.getCacheKey(), filterPresorted.getCacheKey())); + + filterUnsorted = inFilter("column", ColumnType.LONG, Arrays.asList(2L, -2L, 1L, null, 3L)); + filterDifferent = inFilter("column", ColumnType.LONG, Arrays.asList(2L, -2L, 1L, 3L)); + filterPresorted = new TypedInFilter( + "column", + ColumnType.LONG, + null, + Arrays.asList(null, -2L, 1L, 2L, 3L), + null + ); + + Assert.assertEquals(filterPresorted, filterUnsorted); + Assert.assertNotEquals(filterDifferent, filterPresorted); + Assert.assertArrayEquals(filterPresorted.getCacheKey(), filterUnsorted.getCacheKey()); + Assert.assertFalse(Arrays.equals(filterDifferent.getCacheKey(), filterPresorted.getCacheKey())); + + filterUnsorted = inFilter("column", ColumnType.DOUBLE, Arrays.asList(2.2, -2.2, 1.1, null, 3.3)); + filterDifferent = inFilter("column", ColumnType.DOUBLE, Arrays.asList(2.2, -2.2, 1.1, 3.3)); + filterPresorted = new TypedInFilter( + "column", + ColumnType.DOUBLE, + null, + Arrays.asList(null, -2.2, 1.1, 2.2, 3.3), + null + ); + + Assert.assertEquals(filterPresorted, filterUnsorted); + Assert.assertNotEquals(filterDifferent, filterPresorted); + Assert.assertArrayEquals(filterPresorted.getCacheKey(), filterUnsorted.getCacheKey()); + Assert.assertFalse(Arrays.equals(filterDifferent.getCacheKey(), filterPresorted.getCacheKey())); + + filterUnsorted = inFilter("column", ColumnType.FLOAT, Arrays.asList(2.2f, -2.2f, 1.1f, null, 3.3f)); + filterDifferent = inFilter("column", ColumnType.FLOAT, Arrays.asList(2.2f, -2.2f, 1.1f, 3.3f)); + filterPresorted = new TypedInFilter( + "column", + ColumnType.FLOAT, + null, + Arrays.asList(null, -2.2f, 1.1f, 2.2f, 3.3f), + null + ); + + Assert.assertEquals(filterPresorted, filterUnsorted); + Assert.assertNotEquals(filterDifferent, filterPresorted); + Assert.assertArrayEquals(filterPresorted.getCacheKey(), filterUnsorted.getCacheKey()); + Assert.assertFalse(Arrays.equals(filterDifferent.getCacheKey(), filterPresorted.getCacheKey())); + } + + @Test + public void testInvalidParameters() + { + if (NullHandling.replaceWithDefault()) { + Throwable t = Assert.assertThrows( + DruidException.class, + () -> new TypedInFilter("column", ColumnType.STRING, Collections.emptyList(), null, null).toFilter() + ); + Assert.assertEquals("Invalid IN filter, typed in filter only supports SQL compatible null handling mode, set druid.generic.useDefaultValue=false to use this filter", t.getMessage()); + } + + Assume.assumeTrue(NullHandling.sqlCompatible()); + Throwable t = Assert.assertThrows( + DruidException.class, + () -> new TypedInFilter(null, ColumnType.STRING, null, null, null) + ); + Assert.assertEquals("Invalid IN filter, column cannot be null", t.getMessage()); + t = Assert.assertThrows( + DruidException.class, + () -> new TypedInFilter("dim0", null, null, null, null) + ); + Assert.assertEquals("Invalid IN filter on column [dim0], matchValueType cannot be null", t.getMessage()); + t = Assert.assertThrows( + DruidException.class, + () -> new TypedInFilter("dim0", ColumnType.STRING, null, null, null) + ); + Assert.assertEquals( + "Invalid IN filter on column [dim0], exactly one of values or sortedValues must be non-null", + t.getMessage() + ); + } + + @Test + public void testGetDimensionRangeSet() + { + Assume.assumeTrue(NullHandling.sqlCompatible()); + TypedInFilter filter = inFilter("x", ColumnType.STRING, Arrays.asList(null, "a", "b", "c")); + TypedInFilter filter2 = inFilter("x", ColumnType.STRING, Arrays.asList("a", "b", null, "c")); + + Assert.assertEquals(filter.getDimensionRangeSet("x"), filter2.getDimensionRangeSet("x")); + RangeSet range = filter.getDimensionRangeSet("x"); + Assert.assertTrue(range.contains("b")); + + filter = inFilter("x", ColumnType.LONG, Arrays.asList(null, 1L, 2L, 3L)); + filter2 = inFilter("x", ColumnType.LONG, Arrays.asList(3L, 1L, null, 2L)); + Assert.assertEquals(filter.getDimensionRangeSet("x"), filter2.getDimensionRangeSet("x")); + range = filter.getDimensionRangeSet("x"); + Assert.assertTrue(range.contains("2")); + + filter = inFilter("x", ColumnType.DOUBLE, Arrays.asList(null, 1.1, 2.2, 3.3)); + filter2 = inFilter("x", ColumnType.DOUBLE, Arrays.asList(3.3, 1.1, null, 2.2)); + range = filter.getDimensionRangeSet("x"); + Assert.assertEquals(filter.getDimensionRangeSet("x"), filter2.getDimensionRangeSet("x")); + Assert.assertTrue(range.contains("2.2")); + + filter = inFilter("x", ColumnType.FLOAT, Arrays.asList(null, 1.1f, 2.2f, 3.3f)); + filter2 = inFilter("x", ColumnType.FLOAT, Arrays.asList(3.3f, 1.1f, null, 2.2f)); + range = filter.getDimensionRangeSet("x"); + Assert.assertEquals(filter.getDimensionRangeSet("x"), filter2.getDimensionRangeSet("x")); + Assert.assertTrue(range.contains("2.2")); + } + + @Test + public void testRequiredColumnRewrite() + { + Assume.assumeTrue(NullHandling.sqlCompatible()); + TypedInFilter filter = inFilter("dim0", ColumnType.STRING, Arrays.asList("a", "c")); + TypedInFilter filter2 = inFilter("dim1", ColumnType.STRING, Arrays.asList("a", "c")); + + 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 testEquals() + { + Assume.assumeTrue(NullHandling.sqlCompatible()); + EqualsVerifier.forClass(TypedInFilter.class).usingGetClass() + .withNonnullFields( + "column", + "matchValueType", + "unsortedValues", + "sortedMatchValues", + "optimizedFilterIncludeUnknown", + "optimizedFilterNoIncludeUnknown" + ) + .withPrefabValues(ColumnType.class, ColumnType.STRING, ColumnType.DOUBLE) + .withPrefabValues( + Supplier.class, + Suppliers.ofInstance(ImmutableList.of("a", "b")), + Suppliers.ofInstance(ImmutableList.of("b", "c")) + ) + .withIgnoredFields( + "unsortedValues", + "sortedUtf8MatchValueBytes", + "predicateFactorySupplier", + "cacheKeySupplier", + "optimizedFilterIncludeUnknown", + "optimizedFilterNoIncludeUnknown" + ) + .verify(); + } + } + + public static class LegacyInDimFilterNonParameterizedTests extends InitializedNullHandlingTest + { + @Test + public void testRequiredColumnRewrite() + { + InDimFilter filter = (InDimFilter) legacyInFilter("dim0", "a", "c").toFilter(); + InDimFilter filter2 = (InDimFilter) legacyInFilter("dim1", "a", "c").toFilter(); + + 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 testEuals() + { + EqualsVerifier.forClass(InDimFilter.class) + .usingGetClass() + .withNonnullFields("dimension", "values") + .withIgnoredFields( + "cacheKeySupplier", + "predicateFactory", + "optimizedFilterIncludeUnknown", + "optimizedFilterNoIncludeUnknown", + "valuesUtf8" + ) + .verify(); + } + + @Test + public void testEqualsForInFilterDruidPredicateFactory() + { + EqualsVerifier.forClass(InDimFilter.InFilterDruidPredicateFactory.class) + .usingGetClass() + .withNonnullFields("values") + .withIgnoredFields( + "longPredicateSupplier", + "floatPredicateSupplier", + "doublePredicateSupplier", + "stringPredicateSupplier" + ) + .verify(); + } + } + + private static TypedInFilter inFilter(String columnName, ColumnType matchValueType, List values) + { + return new TypedInFilter( + columnName, + matchValueType, + values, + null, + null + ); + } + + private static InDimFilter legacyInFilter(String dim, String value, String... values) + { + return new InDimFilter(dim, Lists.asList(value, values), null); + } + + private static InDimFilter legacyInFilterWithFn(String dim, ExtractionFn fn, String value, String... values) + { + return new InDimFilter(dim, Lists.asList(value, values), fn); + } +} 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 bfe0248d9295..c51b94fc59a5 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 @@ -45,6 +45,10 @@ import java.util.Arrays; import java.util.Map; +/** + * Classic {@link SelectorFilter} test. Consider adding tests to {@link EqualityFilterTests} in addition to, or + * instead of here. + */ @RunWith(Parameterized.class) public class SelectorFilterTest extends BaseFilterTest { 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 be01d4beb3b6..d3e73cf3d681 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 @@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.OperandTypes; import org.apache.calcite.sql.type.ReturnTypes; import org.apache.calcite.sql.type.SqlTypeFamily; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExprEval; @@ -36,6 +37,7 @@ import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.NullFilter; import org.apache.druid.query.filter.OrDimFilter; +import org.apache.druid.query.filter.TypedInFilter; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.sql.calcite.expression.DruidExpression; @@ -46,6 +48,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; public class ArrayOverlapOperatorConversion extends BaseExpressionDimFilterOperatorConversion @@ -130,12 +133,19 @@ public DimFilter toDruidFilter( if (plannerContext.isUseBoundsAndSelectors()) { return newSelectorDimFilter(simpleExtractionExpr.getSimpleExtraction(), Evals.asString(arrayElements[0])); } else { - final String column = simpleExtractionExpr.isDirectColumnAccess() - ? simpleExtractionExpr.getSimpleExtraction().getColumn() - : virtualColumnRegistry.getOrCreateVirtualColumnForExpression( - simpleExtractionExpr, - simpleExtractionExpr.getDruidType() - ); + final String column; + if (simpleExtractionExpr.isDirectColumnAccess()) { + column = simpleExtractionExpr.getDirectColumn(); + } else { + if (virtualColumnRegistry == null) { + // fall back to expression filter + return toExpressionFilter(plannerContext, druidExpressions); + } + column = virtualColumnRegistry.getOrCreateVirtualColumnForExpression( + simpleExtractionExpr, + simpleExtractionExpr.getDruidType() + ); + } final Object elementValue = arrayElements[0]; if (elementValue == null) { return NullFilter.forColumn(column); @@ -148,17 +158,27 @@ public DimFilter toDruidFilter( ); } } else { - final InDimFilter.ValuesSet valuesSet = InDimFilter.ValuesSet.create(); - for (final Object arrayElement : arrayElements) { - valuesSet.add(Evals.asString(arrayElement)); - } + if (plannerContext.isUseBoundsAndSelectors() || NullHandling.replaceWithDefault() || !simpleExtractionExpr.isDirectColumnAccess()) { + final InDimFilter.ValuesSet valuesSet = InDimFilter.ValuesSet.create(); + for (final Object arrayElement : arrayElements) { + valuesSet.add(Evals.asString(arrayElement)); + } - return new InDimFilter( - simpleExtractionExpr.getSimpleExtraction().getColumn(), - valuesSet, - simpleExtractionExpr.getSimpleExtraction().getExtractionFn(), - null - ); + return new InDimFilter( + simpleExtractionExpr.getSimpleExtraction().getColumn(), + valuesSet, + simpleExtractionExpr.getSimpleExtraction().getExtractionFn(), + null + ); + } else { + return new TypedInFilter( + simpleExtractionExpr.getSimpleExtraction().getColumn(), + ExpressionType.toColumnType((ExpressionType) exprEval.type().getElementType()), + Arrays.asList(arrayElements), + null, + null + ); + } } } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CollectComparisons.java b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CollectComparisons.java index 52b518ff8cfb..e67600bb6da5 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CollectComparisons.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/filtration/CollectComparisons.java @@ -23,10 +23,10 @@ import it.unimi.dsi.fastutil.ints.IntSet; import it.unimi.dsi.fastutil.objects.ObjectIntPair; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.query.filter.InDimFilter; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collection; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -37,7 +37,7 @@ * comparisons with the same {@link CollectionKey} can potentially become a single {@link CollectedType}. * For example: x = 'a', x = 'b' can become x IN ('a', 'b'). */ -public abstract class CollectComparisons +public abstract class CollectComparisons> { /** * List of {@link BaseType} that were ORed together. @@ -94,7 +94,7 @@ public List collect() // Emit a collected comparison (e.g. IN filters) for each collection. for (Map.Entry>, List>> entry : collectMap.entrySet()) { final List> comparisonList = entry.getValue(); - final InDimFilter.ValuesSet values = new InDimFilter.ValuesSet(); + final CollectionType values = makeCollection(); for (ObjectIntPair subEntry : comparisonList) { final ComparisonType selector = subEntry.first(); @@ -153,6 +153,8 @@ public List collect() @Nullable protected abstract Pair> getCollectibleComparison(BaseType expr); + protected abstract CollectionType makeCollection(); + /** * Given a comparison, returns its collection key, which will be used to group it together with like comparisons. * This method will be called on objects returned by {@link #getCollectibleComparison(Object)}. If this method returns @@ -164,14 +166,14 @@ public List collect() /** * Given a comparison, returns the strings that it matches. */ - protected abstract Set getMatchValues(ComparisonType comparison); + protected abstract Set getMatchValues(ComparisonType comparison); /** * Given a set of strings from {@link #getMatchValues(Object)} from various comparisons, returns a single collected * comparison that matches all those strings. */ @Nullable - protected abstract CollectedType makeCollectedComparison(CollectionKey key, InDimFilter.ValuesSet values); + protected abstract CollectedType makeCollectedComparison(CollectionKey key, CollectionType values); /** * Given a list of expressions, returns an AND expression with those exprs as children. Only called if 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 5f457096a31b..413d75fbc6dc 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 @@ -20,17 +20,17 @@ package org.apache.druid.sql.calcite.filtration; import com.google.common.collect.Lists; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Pair; -import org.apache.druid.math.expr.ExprEval; -import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.math.expr.Evals; 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.query.filter.InDimFilter; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; +import org.apache.druid.query.filter.TypedInFilter; 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; @@ -66,7 +66,11 @@ public DimFilter process(DimFilter filter) children = new CollectSelectors(children, sourceRowSignature).collect(); // Process "equality" filters, which are used when "sqlUseBoundAndSelectors" is false. - children = new CollectEqualities(children).collect(); + if (NullHandling.sqlCompatible()) { + children = new CollectEqualities(children).collect(); + } else { + children = new CollectEqualitiesDefaultValueMode(children).collect(); + } if (!children.equals(((OrDimFilter) filter).getFields())) { return children.size() == 1 ? children.get(0) : new OrDimFilter(children); @@ -131,7 +135,7 @@ private static Pair> splitAnd( * Helper for collecting {@link SelectorDimFilter} into {@link InDimFilter}. */ private static class CollectSelectors - extends CollectComparisons + extends CollectComparisons { private final RowSignature sourceRowSignature; @@ -155,6 +159,12 @@ protected Pair> getCollectibleComparison(DimF ); } + @Override + protected InDimFilter.ValuesSet makeCollection() + { + return new InDimFilter.ValuesSet(); + } + @Nullable @Override protected BoundRefKey getCollectionKey(SelectorDimFilter selector) @@ -195,7 +205,8 @@ protected DimFilter makeAnd(List exprs) /** * Helper for collecting {@link EqualityFilter} into {@link InDimFilter}. */ - private static class CollectEqualities extends CollectComparisons + private static class CollectEqualities + extends CollectComparisons> { public CollectEqualities(final List orExprs) { @@ -216,27 +227,81 @@ protected Pair> getCollectibleComparison(DimFilt ); } + @Override + protected List makeCollection() + { + return new ArrayList<>(); + } + @Nullable @Override protected RangeRefKey getCollectionKey(EqualityFilter selector) { - if (!selector.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 - return null; + return RangeRefKey.from(selector); + } + + @Override + protected Set getMatchValues(EqualityFilter selector) + { + return Collections.singleton(selector.getMatchValue()); + } + + @Nullable + @Override + protected TypedInFilter makeCollectedComparison(RangeRefKey rangeRefKey, List values) + { + if (values.size() > 1) { + return new TypedInFilter(rangeRefKey.getColumn(), rangeRefKey.getMatchValueType(), values, null, null); } + return null; + } + + @Override + protected DimFilter makeAnd(List exprs) + { + return new AndDimFilter(exprs); + } + } + private static class CollectEqualitiesDefaultValueMode + extends CollectComparisons + { + public CollectEqualitiesDefaultValueMode(final List orExprs) + { + super(orExprs); + } + + @Nullable + @Override + protected Pair> getCollectibleComparison(DimFilter filter) + { + return ConvertSelectorsToIns.splitAnd( + filter, + EqualityFilter.class, + + // Prefer extracting nonnull vs null comparisons when ANDed, as nonnull comparisons are more likely to + // find companions in other ORs. + Comparator.comparing(equality -> equality.getMatchValue() == null ? 0 : 1) + ); + } + + @Override + protected InDimFilter.ValuesSet makeCollection() + { + return new InDimFilter.ValuesSet(); + } + + @Nullable + @Override + protected RangeRefKey getCollectionKey(EqualityFilter selector) + { return RangeRefKey.from(selector); } @Override protected Set getMatchValues(EqualityFilter selector) { - return Collections.singleton( - ExprEval.ofType(ExpressionType.fromColumnType(selector.getMatchValueType()), selector.getMatchValue()) - .castTo(ExpressionType.STRING) - .asString() - ); + return Collections.singleton(Evals.asString(selector.getMatchValue())); } @Nullable @@ -244,10 +309,11 @@ protected Set getMatchValues(EqualityFilter selector) protected InDimFilter makeCollectedComparison(RangeRefKey rangeRefKey, InDimFilter.ValuesSet values) { if (values.size() > 1) { + // skip non-string equality filters since InDimFilter uses a sorted string set, which is a different sort + // than numbers or other types might use return new InDimFilter(rangeRefKey.getColumn(), values, null, null); - } else { - return null; } + return null; } @Override diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java b/sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java index f53cc3e282a0..95ad2b11334b 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/rule/ReverseLookupRule.java @@ -310,7 +310,7 @@ private RexNode visitComparison(final RexCall call) * Collect and reverse a set of lookups that appear as children to OR. */ private class CollectReverseLookups - extends CollectComparisons + extends CollectComparisons { private final RexBuilder rexBuilder; @@ -335,6 +335,12 @@ protected Pair> getCollectibleComparison(RexNode expr) } } + @Override + protected InDimFilter.ValuesSet makeCollection() + { + return new InDimFilter.ValuesSet(); + } + @Nullable @Override protected ReverseLookupKey getCollectionKey(RexCall call) 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 d83a9da4c8dd..4c18432a13d8 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 @@ -70,6 +70,7 @@ 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.filter.TypedInFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.having.DimFilterHavingSpec; import org.apache.druid.query.lookup.LookupExtractorFactoryContainerProvider; @@ -369,11 +370,31 @@ public static IsTrueDimFilter istrue(DimFilter filter) return new IsTrueDimFilter(filter); } - public static InDimFilter in(String dimension, Collection values, ExtractionFn extractionFn) + public static DimFilter in(String dimension, Collection values) { + if (NullHandling.sqlCompatible()) { + return in(dimension, ColumnType.STRING, new ArrayList<>(values)); + } + return new InDimFilter(dimension, values, null); + } + + public static DimFilter in(String dimension, Collection values, ExtractionFn extractionFn) + { + if (NullHandling.sqlCompatible() && extractionFn == null) { + return in(dimension, ColumnType.STRING, new ArrayList<>(values)); + } return new InDimFilter(dimension, values, extractionFn); } + public static DimFilter in(String dimension, ColumnType matchValueType, List values) + { + if (NullHandling.sqlCompatible()) { + return new TypedInFilter(dimension, matchValueType, values, null, null); + } + Set set = values.stream().map(Evals::asString).collect(Collectors.toSet()); + return in(dimension, set, null); + } + public static DimFilter isNull(final String fieldName) { return isNull(fieldName, null); 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 d7b0a1d45ee1..b870e45967b2 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 @@ -702,7 +702,7 @@ public void testArrayOverlapFilter() newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE3) .intervals(querySegmentSpec(Filtration.eternity())) - .filters(new InDimFilter("dim3", ImmutableList.of("a", "b"), null)) + .filters(in("dim3", ImmutableList.of("a", "b"))) .columns("dim3") .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .limit(5) @@ -4333,7 +4333,7 @@ public void testUnnestTwiceWithFiltersAndExpressions() "string_to_array(\"dim1\",'\\u005C.')", ColumnType.STRING_ARRAY ), - in("j0.unnest", ImmutableList.of("1", "2"), null) + in("j0.unnest", ImmutableList.of("1", "2")) ), expressionVirtualColumn( "_j0.unnest", @@ -5249,7 +5249,7 @@ public void testUnnestWithInFiltersWithExpressionInInnerQuery() .dataSource(UnnestDataSource.create( FilteredDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), - new InDimFilter("dim2", ImmutableList.of("a", "b"), null) + in("dim2", ImmutableList.of("a", "b")) ), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), null @@ -5385,7 +5385,7 @@ public void testUnnestWithFiltersInsideAndOutside1() ), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), or( - in("j0.unnest", ImmutableList.of("a", "c"), null), + in("j0.unnest", ImmutableList.of("a", "c")), new LikeDimFilter("j0.unnest", "_", null, null) ) )) @@ -5427,7 +5427,7 @@ public void testUnnestWithFiltersOutside() ), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), or( - in("j0.unnest", ImmutableList.of("a", "c"), null), + in("j0.unnest", ImmutableList.of("a", "c")), new LikeDimFilter("j0.unnest", "_", null, null) ) )) @@ -5462,7 +5462,7 @@ public void testUnnestWithInFilters() .dataSource(UnnestDataSource.create( FilteredDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), - new InDimFilter("dim2", ImmutableList.of("a", "b", "ab", "abc"), null) + in("dim2", ImmutableList.of("a", "b", "ab", "abc")) ), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), null @@ -5604,7 +5604,7 @@ public void testUnnestWithJoinOnTheLeft() .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) - .filters(new InDimFilter("dim2", ImmutableList.of("a", "b", "ab", "abc"), null)) + .filters(in("dim2", ImmutableList.of("a", "b", "ab", "abc"))) .columns("dim2") .context(QUERY_CONTEXT_UNNEST) .build() @@ -5767,7 +5767,7 @@ public void testUnnestWithINFiltersWithLeftRewrite() range("m1", ColumnType.LONG, null, 10L, false, true) ), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - new InDimFilter("j0.unnest", ImmutableSet.of("a", "b"), null) + in("j0.unnest", ImmutableSet.of("a", "b")) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -5797,7 +5797,7 @@ public void testUnnestWithINFiltersWithNoLeftRewrite() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "array(\"dim4\",\"dim5\")", ColumnType.STRING_ARRAY), - new InDimFilter("j0.unnest", ImmutableSet.of("a", "b"), null) + in("j0.unnest", ImmutableSet.of("a", "b")) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -5830,7 +5830,7 @@ public void testUnnestWithInvalidINFiltersOnUnnestedColumn() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - new InDimFilter("j0.unnest", ImmutableSet.of("foo", "bar"), null) + in("j0.unnest", ImmutableSet.of("foo", "bar")) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -5958,7 +5958,7 @@ public void testUnnestWithSelectorFiltersOnVirtualStringColumn() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "array(\"dim4\",\"dim5\")", ColumnType.STRING_ARRAY), - new InDimFilter("j0.unnest", ImmutableSet.of("a", "ab"), null) + in("j0.unnest", ImmutableSet.of("a", "ab")) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -6061,7 +6061,7 @@ public void testUnnestWithMultipleAndFiltersOnSelectedUnnestedColumns() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - new InDimFilter("j0.unnest", ImmutableSet.of("a", "b"), null) + in("j0.unnest", ImmutableSet.of("a", "b")) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -6091,7 +6091,7 @@ public void testUnnestWithMultipleOrFiltersOnUnnestedColumns() .dataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - new InDimFilter("j0.unnest", ImmutableSet.of("b", "d"), null) + in("j0.unnest", ImmutableSet.of("b", "d")) )) .intervals(querySegmentSpec(Filtration.eternity())) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) @@ -6211,7 +6211,7 @@ public void testUnnestWithMultipleOrFiltersOnSelectedVirtualColumns() .context(QUERY_CONTEXT_UNNEST) .filters( or( - new InDimFilter("j0.unnest", ImmutableSet.of("a", "aa"), null), + in("j0.unnest", ImmutableSet.of("a", "aa")), range("m1", ColumnType.LONG, null, 2L, false, true) ) ) @@ -6453,7 +6453,7 @@ public void testUnnestWithGroupByHavingWithWhereOnAggCol() .setDataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - new InDimFilter("j0.unnest", ImmutableSet.of("a", "c"), null) + in("j0.unnest", ImmutableSet.of("a", "c")) )) .setInterval(querySegmentSpec(Filtration.eternity())) .setContext(QUERY_CONTEXT_UNNEST) @@ -6484,7 +6484,7 @@ public void testUnnestWithGroupByHavingWithWhereOnUnnestCol() .setDataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.DATASOURCE3), expressionVirtualColumn("j0.unnest", "\"dim3\"", ColumnType.STRING), - new InDimFilter("j0.unnest", ImmutableSet.of("a", "c"), null) + in("j0.unnest", ImmutableSet.of("a", "c")) )) .setInterval(querySegmentSpec(Filtration.eternity())) .setContext(QUERY_CONTEXT_UNNEST) @@ -6514,13 +6514,7 @@ public void testUnnestWithGroupByWithWhereOnUnnestArrayCol() .setDataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.ARRAYS_DATASOURCE), expressionVirtualColumn("j0.unnest", "\"arrayLongNulls\"", ColumnType.LONG_ARRAY), - NullHandling.sqlCompatible() - ? or( - equality("j0.unnest", 1L, ColumnType.LONG), - equality("j0.unnest", 2L, ColumnType.LONG), - equality("j0.unnest", 3L, ColumnType.LONG) - ) - : in("j0.unnest", ImmutableList.of("1", "2", "3"), null) + in("j0.unnest", ColumnType.LONG, ImmutableList.of(1L, 2L, 3L)) )) .setInterval(querySegmentSpec(Filtration.eternity())) .setContext(QUERY_CONTEXT_UNNEST) @@ -6551,13 +6545,7 @@ public void testUnnestWithGroupByHavingWithWhereOnUnnestArrayCol() .setDataSource(UnnestDataSource.create( new TableDataSource(CalciteTests.ARRAYS_DATASOURCE), expressionVirtualColumn("j0.unnest", "\"arrayLongNulls\"", ColumnType.LONG_ARRAY), - NullHandling.sqlCompatible() - ? or( - equality("j0.unnest", 1L, ColumnType.LONG), - equality("j0.unnest", 2L, ColumnType.LONG), - equality("j0.unnest", 3L, ColumnType.LONG) - ) - : in("j0.unnest", ImmutableList.of("1", "2", "3"), null) + in("j0.unnest", ColumnType.LONG, ImmutableList.of(1L, 2L, 3L)) )) .setInterval(querySegmentSpec(Filtration.eternity())) .setContext(QUERY_CONTEXT_UNNEST) @@ -6890,12 +6878,9 @@ public void testUnnestWithTimeFilterInsideSubquery() ) .intervals(querySegmentSpec(Filtration.eternity())) .filters( - NullHandling.sqlCompatible() ? - or( - equality("m1", 1.0f, ColumnType.FLOAT), - equality("m1", 2.0f, ColumnType.FLOAT) - ) : - new InDimFilter("m1", ImmutableList.of("1", "2"), null) + NullHandling.sqlCompatible() + ? in("m1", ColumnType.FLOAT, ImmutableList.of(1.0f, 2.0f)) + : in("m1", ImmutableList.of("1", "2")) ) .resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST) .legacy(false) 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 93ce1a7d102d..bf631da78e51 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,7 +63,6 @@ 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.InDimFilter; import org.apache.druid.query.filter.LikeDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.ResultRow; @@ -4037,7 +4036,7 @@ public void testTwoSemiJoinsSimultaneously(Map queryContext) ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(in("dim1", ImmutableList.of("abc", "def"), null)) + .setDimFilter(in("dim1", ImmutableList.of("abc", "def"))) .setDimensions(dimensions(new DefaultDimensionSpec("dim1", "d0", ColumnType.STRING))) .setAggregatorSpecs(aggregators(new CountAggregatorFactory("a0"))) .setContext(queryContext) @@ -4134,7 +4133,7 @@ public void testSemiAndAntiJoinSimultaneouslyUsingWhereInSubquery(Map qu ColumnType.STRING ) ) - .filters(new InDimFilter( - "channel", - new HashSet<>(Arrays.asList( - "abc", - "xyz" - )) - )) + .filters(in("channel", Arrays.asList("abc", "xyz"))) .context(QUERY_CONTEXT_DEFAULT) .build() ), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java index 54adbc7d9001..e172470279cc 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteUnionQueryTest.java @@ -59,7 +59,7 @@ public void testUnionAllDifferentTablesWithMapping() ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(in("dim2", ImmutableList.of("def", "a"), null)) + .setDimFilter(in("dim2", ImmutableList.of("def", "a"))) .setDimensions( new DefaultDimensionSpec("dim1", "d0"), new DefaultDimensionSpec("dim2", "d1") @@ -101,7 +101,7 @@ public void testJoinUnionAllDifferentTablesWithMapping() ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(in("dim2", ImmutableList.of("def", "a"), null)) + .setDimFilter(in("dim2", ImmutableList.of("def", "a"))) .setDimensions( new DefaultDimensionSpec("dim1", "d0"), new DefaultDimensionSpec("dim2", "d1") @@ -167,7 +167,7 @@ public void testUnionAllTablesColumnTypeMismatchFloatLong() ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(in("dim2", ImmutableList.of("en", "a"), null)) + .setDimFilter(in("dim2", ImmutableList.of("en", "a"))) .setDimensions( new DefaultDimensionSpec("dim1", "d0"), new DefaultDimensionSpec("dim2", "d1") @@ -272,7 +272,7 @@ public void testUnionAllSameTableTwice() ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(in("dim2", ImmutableList.of("def", "a"), null)) + .setDimFilter(in("dim2", ImmutableList.of("def", "a"))) .setDimensions( new DefaultDimensionSpec("dim1", "d0"), new DefaultDimensionSpec("dim2", "d1") @@ -314,7 +314,7 @@ public void testUnionAllSameTableTwiceWithSameMapping() ) .setInterval(querySegmentSpec(Filtration.eternity())) .setGranularity(Granularities.ALL) - .setDimFilter(in("dim2", ImmutableList.of("def", "a"), null)) + .setDimFilter(in("dim2", ImmutableList.of("def", "a"))) .setDimensions( new DefaultDimensionSpec("dim1", "d0"), new DefaultDimensionSpec("dim2", "d1")