From d0f64608eb1c31a0c65a9cc92ef41ce06a5032ed Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 12 Oct 2023 00:06:23 -0700 Subject: [PATCH] sql compatible three-valued logic native filters (#15058) * sql compatible tri-state native logical filters when druid.expressions.useStrictBooleans=true and druid.generic.useDefaultValueForNull=false, and new druid.generic.useThreeValueLogicForNativeFilters=true * log.warn if non-default configurations are used to guide operators towards SQL complaint behavior --- docs/configuration/index.md | 3 +- docs/querying/filters.md | 2 + docs/querying/sql-data-types.md | 12 +- ...TypeMapVirtualColumnDimensionSelector.java | 8 +- ...TypeMapVirtualColumnDimensionSelector.java | 15 +- .../segment/MapVirtualColumnGroupByTest.java | 140 ++++++- .../sql/DoublesSketchSqlAggregatorTest.java | 2 +- .../druid/query/filter/BloomDimFilter.java | 8 + .../query/filter/BloomDimFilterTest.java | 32 +- .../common/SortMergeJoinFrameProcessor.java | 6 +- .../scan/ExternalColumnSelectorFactory.java | 6 +- .../apache/druid/msq/exec/MSQSelectTest.java | 6 +- .../druid/common/config/NullHandling.java | 18 +- .../config/NullValueHandlingConfig.java | 39 +- .../druid/frame/field/StringFieldReader.java | 6 +- .../processor/MultiColumnSelectorFactory.java | 6 +- .../columnar/StringFrameColumnReader.java | 6 +- .../frame/segment/FrameFilteredOffset.java | 6 +- .../math/expr/ExpressionProcessingConfig.java | 15 + .../query/aggregation/FilteredAggregator.java | 2 +- .../aggregation/FilteredBufferAggregator.java | 2 +- .../aggregation/FilteredVectorAggregator.java | 4 +- .../ForwardingFilteredDimensionSelector.java | 82 ++-- .../PredicateFilteredDimensionSelector.java | 21 +- .../query/filter/DruidPredicateFactory.java | 17 + .../druid/query/filter/EqualityFilter.java | 13 +- .../org/apache/druid/query/filter/Filter.java | 8 +- .../druid/query/filter/InDimFilter.java | 8 + .../druid/query/filter/NotDimFilter.java | 5 + .../apache/druid/query/filter/NullFilter.java | 7 + .../druid/query/filter/RangeFilter.java | 4 +- .../filter/SelectorPredicateFactory.java | 8 + .../StringPredicateDruidPredicateFactory.java | 99 +++++ .../druid/query/filter/ValueMatcher.java | 12 +- .../vector/ArrayVectorValueMatcher.java | 7 +- .../vector/BooleanVectorValueMatcher.java | 17 +- .../vector/DoubleVectorValueMatcher.java | 25 +- .../vector/FloatVectorValueMatcher.java | 25 +- .../filter/vector/LongVectorValueMatcher.java | 24 +- .../MultiValueStringVectorValueMatcher.java | 36 +- .../vector/ObjectVectorValueMatcher.java | 16 +- .../SingleValueStringVectorValueMatcher.java | 59 ++- .../StringObjectVectorValueMatcher.java | 13 +- .../filter/vector/VectorValueMatcher.java | 284 ++++++++++++- .../vector/VectorValueMatcherFactory.java | 34 -- .../epinephelinae/RowBasedGrouperHelper.java | 6 +- .../groupby/having/DimFilterHavingSpec.java | 2 +- .../LazilyDecoratedRowsAndColumns.java | 2 +- .../apache/druid/query/scan/ConcatCursor.java | 6 +- .../query/search/UseIndexesStrategy.java | 2 +- .../BaseSingleValueDimensionSelector.java | 17 +- .../segment/ConstantDimensionSelector.java | 10 +- .../ConstantMultiValueDimensionSelector.java | 16 +- .../DictionaryEncodedColumnIndexer.java | 4 +- .../DictionaryEncodedColumnMerger.java | 4 +- .../druid/segment/DimensionSelector.java | 19 +- .../druid/segment/DimensionSelectorUtils.java | 96 +++-- .../apache/druid/segment/FilteredOffset.java | 17 +- .../apache/druid/segment/IndexMergerV9.java | 1 + .../RowBasedColumnSelectorFactory.java | 23 +- .../apache/druid/segment/RowBasedCursor.java | 6 +- .../SingleScanTimeDimensionSelector.java | 17 +- .../druid/segment/StringDimensionIndexer.java | 60 ++- .../druid/segment/UnnestDimensionCursor.java | 19 +- .../StringUtf8DictionaryEncodedColumn.java | 60 ++- ...Matcher.java => AllFalseValueMatcher.java} | 15 +- ...eMatcher.java => AllTrueValueMatcher.java} | 13 +- ...tcher.java => AllUnknownValueMatcher.java} | 27 +- .../druid/segment/filter/AndFilter.java | 18 +- .../druid/segment/filter/BoundFilter.java | 22 +- .../filter/ColumnComparisonFilter.java | 9 +- .../segment/filter/ConstantMatcherType.java | 81 ++++ .../filter/DimensionPredicateFilter.java | 8 + .../segment/filter/ExpressionFilter.java | 47 ++- .../druid/segment/filter/FalseFilter.java | 7 +- .../apache/druid/segment/filter/Filters.java | 11 +- .../druid/segment/filter/LikeFilter.java | 4 +- .../druid/segment/filter/NotFilter.java | 31 +- .../apache/druid/segment/filter/OrFilter.java | 18 +- .../filter/PredicateValueMatcherFactory.java | 34 +- .../druid/segment/filter/SpatialFilter.java | 11 +- .../StringConstantValueMatcherFactory.java | 1 + .../druid/segment/filter/TrueFilter.java | 5 +- .../druid/segment/filter/ValueMatchers.java | 386 ++++++++++++----- .../IncrementalIndexStorageAdapter.java | 10 +- .../index/AllFalseBitmapColumnIndex.java | 22 +- .../index/AllTrueBitmapColumnIndex.java | 2 +- .../index/AllUnknownBitmapColumnIndex.java | 64 +++ .../segment/index/BitmapColumnIndex.java | 18 +- .../IndexedStringDruidPredicateIndexes.java | 11 + ...ndexedUtf8LexicographicalRangeIndexes.java | 27 +- .../index/IndexedUtf8ValueIndexes.java | 44 +- .../index/SimpleImmutableBitmapIndex.java | 5 +- .../SimpleImmutableBitmapIterableIndex.java | 20 +- .../join/PossiblyNullDimensionSelector.java | 6 +- .../druid/segment/join/PostJoinCursor.java | 2 +- .../table/IndexedTableDimensionSelector.java | 6 +- .../NestedFieldColumnIndexSupplier.java | 155 ++++++- .../NestedFieldDictionaryEncodedColumn.java | 35 +- .../ScalarDoubleColumnAndIndexSupplier.java | 51 ++- .../ScalarLongColumnAndIndexSupplier.java | 51 ++- .../druid/segment/nested/VariantColumn.java | 38 +- .../nested/VariantColumnAndIndexSupplier.java | 28 +- .../SettableDimensionValueSelector.java | 4 +- .../druid/segment/transform/Transformer.java | 4 +- .../segment/vector/FilteredVectorOffset.java | 5 +- ...ExpressionMultiValueDimensionSelector.java | 21 +- .../segment/virtual/ExpressionSelectors.java | 8 +- .../virtual/ListFilteredVirtualColumn.java | 210 ++++++---- .../virtual/NestedFieldVirtualColumn.java | 6 +- ...EvaluationExpressionDimensionSelector.java | 6 +- .../intset/ImmutableConciseSetTest.java | 20 + .../frame/field/DoubleFieldReaderTest.java | 33 +- .../frame/field/FloatFieldReaderTest.java | 33 +- .../frame/field/LongFieldReaderTest.java | 33 +- .../frame/field/StringFieldReaderTest.java | 17 +- .../aggregation/FilteredAggregatorTest.java | 6 +- .../CardinalityAggregatorTest.java | 6 +- .../StringDimensionSelectorForTest.java | 6 +- .../dimension/TestDimensionSelector.java | 6 +- .../druid/query/filter/InDimFilterTest.java | 6 +- .../filter/SelectorPredicateFactoryTest.java | 2 +- ...alueMatcherColumnProcessorFactoryTest.java | 15 +- .../GroupByTimeseriesQueryRunnerTest.java | 29 ++ .../timeseries/TimeseriesQueryRunnerTest.java | 71 ++-- .../topn/TopNMetricSpecOptimizationsTest.java | 4 +- ...ColumnSelectorColumnIndexSelectorTest.java | 5 +- .../ConstantDimensionSelectorTest.java | 22 +- ...nstantMultiValueDimensionSelectorTest.java | 49 +-- .../apache/druid/segment/IndexBuilder.java | 2 +- .../segment/IndexMergerNullHandlingTest.java | 3 +- .../druid/segment/IndexMergerTestBase.java | 2 +- .../org/apache/druid/segment/ListCursor.java | 4 +- .../druid/segment/filter/BaseFilterTest.java | 44 +- .../druid/segment/filter/BoundFilterTest.java | 11 +- .../filter/DimensionPredicateFilterTest.java | 2 +- .../segment/filter/EqualityFilterTests.java | 387 +++++++++++++++++- ...ExpressionFilterNonStrictBooleansTest.java | 90 ++++ .../segment/filter/ExpressionFilterTest.java | 115 ++++-- .../filter/FilterCnfConversionTest.java | 2 +- .../segment/filter/FilterPartitionTest.java | 8 +- .../druid/segment/filter/InFilterTest.java | 159 ++++++- .../druid/segment/filter/LikeFilterTest.java | 123 +++++- .../druid/segment/filter/NullFilterTest.java | 237 ----------- .../druid/segment/filter/NullFilterTests.java | 343 ++++++++++++++++ .../PredicateValueMatcherFactoryTest.java | 60 +-- .../segment/filter/RangeFilterTests.java | 170 +++++++- .../segment/filter/ValueMatchersTest.java | 32 +- .../nested/NestedDataColumnSupplierTest.java | 50 +-- .../NestedDataColumnSupplierV4Test.java | 48 +-- .../NestedFieldColumnIndexSupplierTest.java | 246 +++++------ .../ScalarDoubleColumnSupplierTest.java | 38 +- .../nested/ScalarLongColumnSupplierTest.java | 38 +- .../ScalarStringColumnSupplierTest.java | 52 +-- .../nested/VariantColumnSupplierTest.java | 16 +- ...tionaryEncodedStringIndexSupplierTest.java | 22 +- .../virtual/DummyStringVirtualColumn.java | 4 +- .../virtual/ExpressionVirtualColumnTest.java | 75 ++-- .../segment/virtual/VirtualColumnsTest.java | 6 +- .../sql/calcite/CalciteArraysQueryTest.java | 4 +- .../sql/calcite/CalciteJoinQueryTest.java | 20 +- .../calcite/CalciteParameterQueryTest.java | 7 +- .../druid/sql/calcite/CalciteQueryTest.java | 25 +- .../expression/ExpressionTestHelper.java | 2 +- 164 files changed, 4392 insertions(+), 1580 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/filter/StringPredicateDruidPredicateFactory.java rename processing/src/main/java/org/apache/druid/segment/filter/{FalseValueMatcher.java => AllFalseValueMatcher.java} (76%) rename processing/src/main/java/org/apache/druid/segment/filter/{TrueValueMatcher.java => AllTrueValueMatcher.java} (75%) rename processing/src/main/java/org/apache/druid/segment/filter/{BooleanValueMatcher.java => AllUnknownValueMatcher.java} (56%) create mode 100644 processing/src/main/java/org/apache/druid/segment/filter/ConstantMatcherType.java create mode 100644 processing/src/main/java/org/apache/druid/segment/index/AllUnknownBitmapColumnIndex.java delete mode 100644 processing/src/test/java/org/apache/druid/segment/filter/NullFilterTest.java create mode 100644 processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 9b6fba9ee63..4c0bb0c2b89 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -798,8 +798,9 @@ Support for 64-bit floating point columns was released in Druid 0.11.0, so if yo Prior to version 0.13.0, Druid string columns treated `''` and `null` values as interchangeable, and numeric columns were unable to represent `null` values, coercing `null` to `0`. Druid 0.13.0 introduced a mode which enabled SQL compatible null handling, allowing string columns to distinguish empty strings from nulls, and numeric columns to contain null rows. |Property|Description|Default| -|---|---|---| +|--------|-----------|-------| |`druid.generic.useDefaultValueForNull`|Set to `false` to store and query data in SQL compatible mode. When set to `true` (legacy mode), `null` values will be stored as `''` for string columns and `0` for numeric columns.|`false`| +|`druid.generic.useThreeValueLogicForNativeFilters`|Set to `true` to use SQL compatible three-value logic when processing native Druid filters when `druid.generic.useDefaultValueForNull=false` and `druid.expressions.useStrictBooleans=true`. When set to `false` Druid uses 2 value logic for filter processing, even when `druid.generic.useDefaultValueForNull=false` and `druid.expressions.useStrictBooleans=true`. See [boolean handling](../querying/sql-data-types.md#boolean-logic) for more details|`true`| |`druid.generic.ignoreNullsForStringCardinality`|When set to `true`, `null` values will be ignored for the built-in cardinality aggregator over string columns. Set to `false` to include `null` values while estimating cardinality of only string columns using the built-in cardinality aggregator. This setting takes effect only when `druid.generic.useDefaultValueForNull` is set to `true` and is ignored in SQL compatibility mode. Additionally, empty strings (equivalent to null) are not counted when this is set to `true`. |`false`| This mode does have a storage size and query performance cost, see [segment documentation](../design/segments.md#handling-null-values) for more details. diff --git a/docs/querying/filters.md b/docs/querying/filters.md index 431a29b556b..9ded2455167 100644 --- a/docs/querying/filters.md +++ b/docs/querying/filters.md @@ -33,6 +33,8 @@ sidebar_label: "Filters" A filter is a JSON object indicating which rows of data should be included in the computation for a query. It’s essentially the equivalent of the WHERE clause in SQL. Filters are commonly applied on dimensions, but can be applied on aggregated metrics, for example, see [Filtered aggregator](./aggregations.md#filtered-aggregator) and [Having filters](./having.md). +By default, Druid uses SQL compatible three-value logic when filtering. See [Boolean logic](./sql-data-types.md#boolean-logic) for more details. + Apache Druid supports the following types of filters. ## Selector filter diff --git a/docs/querying/sql-data-types.md b/docs/querying/sql-data-types.md index 6f01369e844..2b30407bc3b 100644 --- a/docs/querying/sql-data-types.md +++ b/docs/querying/sql-data-types.md @@ -152,14 +152,14 @@ values are treated as zeroes. This was the default prior to Druid 28.0.0. ## Boolean logic -The [`druid.expressions.useStrictBooleans`](../configuration/index.md#expression-processing-configurations) -runtime property controls Druid's boolean logic mode. For the most SQL compliant behavior, set this to `true` (the default). +By default, Druid uses [SQL three-valued logic](https://en.wikipedia.org/wiki/Three-valued_logic#SQL) for filter processing +and boolean expression evaluation. This behavior relies on three settings: -When `druid.expressions.useStrictBooleans = true`, Druid uses three-valued logic for -[expressions](math-expr.md) evaluation, such as `expression` virtual columns or `expression` filters. -However, even in this mode, Druid uses two-valued logic for filter types other than `expression`. +* [`druid.generic.useDefaultValueForNull`](../configuration/index.md#sql-compatible-null-handling) must be set to false (default), a runtime property which allows NULL values to exist in numeric columns and expressions, and string typed columns to distinguish between NULL and the empty string +* [`druid.expressions.useStrictBooleans`](../configuration/index.md#expression-processing-configurations) must be set to true (default), a runtime property controls Druid's boolean logic mode for expressions, as well as coercing all expression boolean values to be represented with a 1 for true and 0 for false +* [`druid.generic.useThreeValueLogicForNativeFilters`](../configuration/index.md#sql-compatible-null-handling) must be set to true (default), a runtime property which decouples three-value logic handling from `druid.generic.useDefaultValueForNull` and `druid.expressions.useStrictBooleans` for backwards compatibility with older versions of Druid that did not fully support SQL compatible null value logic handling -When `druid.expressions.useStrictBooleans = false` (legacy mode), Druid uses two-valued logic. +If any of these settings is configured with a non-default value, Druid will use two-valued logic for non-expression based filters. Expression based filters are controlled independently with `druid.expressions.useStrictBooleans`, which if set to false Druid will use two-valued logic for expressions. ## Nested columns diff --git a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java index 4fc361f3325..841fb192080 100644 --- a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java +++ b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/MapTypeMapVirtualColumnDimensionSelector.java @@ -19,7 +19,7 @@ package org.apache.druid.segment; -import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.IndexedInts; @@ -55,7 +55,7 @@ final class MapTypeMapVirtualColumnDimensionSelector extends MapVirtualColumnDim return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { // Map column doesn't match with any string return false; @@ -70,12 +70,12 @@ final class MapTypeMapVirtualColumnDimensionSelector extends MapVirtualColumnDim } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { return false; } diff --git a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java index 37a00520369..faf1e24a981 100644 --- a/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java +++ b/extensions-contrib/virtual-columns/src/main/java/org/apache/druid/segment/StringTypeMapVirtualColumnDimensionSelector.java @@ -21,6 +21,7 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.IndexedInts; @@ -68,9 +69,10 @@ final class StringTypeMapVirtualColumnDimensionSelector extends MapVirtualColumn return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return Objects.equals(value, getObject()); + final Object rowValue = getObject(); + return (includeUnknown && rowValue == null) || Objects.equals(value, rowValue); } @Override @@ -84,14 +86,17 @@ final class StringTypeMapVirtualColumnDimensionSelector extends MapVirtualColumn } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { + final Predicate predicate = predicateFactory.makeStringPredicate(); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return predicate.apply((String) getObject()); + final String rowValue = (String) getObject(); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + return (matchNull && rowValue == null) || predicate.apply(rowValue); } @Override diff --git a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java index fb1ca371781..ebbee1f8a54 100644 --- a/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java +++ b/extensions-contrib/virtual-columns/src/test/java/org/apache/druid/segment/MapVirtualColumnGroupByTest.java @@ -22,6 +22,7 @@ package org.apache.druid.segment; import com.google.common.collect.ImmutableList; import org.apache.druid.collections.DefaultBlockingPool; import org.apache.druid.collections.StupidPool; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; @@ -34,6 +35,9 @@ import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.TableDataSource; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.dimension.DefaultDimensionSpec; +import org.apache.druid.query.filter.EqualityFilter; +import org.apache.druid.query.filter.InDimFilter; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.groupby.GroupByQuery; import org.apache.druid.query.groupby.GroupByQueryConfig; import org.apache.druid.query.groupby.GroupByQueryQueryToolChest; @@ -41,25 +45,22 @@ import org.apache.druid.query.groupby.GroupByQueryRunnerFactory; import org.apache.druid.query.groupby.GroupingEngine; import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; +import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.SegmentId; import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest { - @Rule - public ExpectedException expectedException = ExpectedException.none(); - private QueryRunner runner; @Before @@ -132,11 +133,14 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest null ); - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("Map column doesn't support getRow()"); - runner.run(QueryPlus.wrap(query)).toList(); + Throwable t = Assert.assertThrows( + UnsupportedOperationException.class, + () -> runner.run(QueryPlus.wrap(query)).toList() + ); + Assert.assertEquals("Map column doesn't support getRow()", t.getMessage()); } + @Test public void testWithSubColumn() { @@ -166,4 +170,124 @@ public class MapVirtualColumnGroupByTest extends InitializedNullHandlingTest Assert.assertEquals(expected, result); } + + @Test + public void testWithSubColumnWithFilter() + { + final GroupByQuery query = new GroupByQuery( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), + VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + new EqualityFilter("params.key3", ColumnType.STRING, "value3", null), + Granularities.ALL, + ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), + ImmutableList.of(new CountAggregatorFactory("count")), + null, + null, + null, + null, + null + ); + + final List result = runner.run(QueryPlus.wrap(query)).toList(); + final List expected = ImmutableList.of( + new MapBasedRow( + DateTimes.of("2011-01-12T00:00:00.000Z"), + MapVirtualColumnTestBase.mapOf("count", 1L, "params.key3", "value3") + ) + ).stream().map(row -> ResultRow.fromLegacyRow(row, query)).collect(Collectors.toList()); + + Assert.assertEquals(expected, result); + } + + @Test + public void testWithSubColumnWithPredicateFilter() + { + final GroupByQuery query = new GroupByQuery( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), + VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + new InDimFilter("params.key3", ImmutableList.of("value1", "value3"), null), + Granularities.ALL, + ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), + ImmutableList.of(new CountAggregatorFactory("count")), + null, + null, + null, + null, + null + ); + + final List result = runner.run(QueryPlus.wrap(query)).toList(); + final List expected = ImmutableList.of( + new MapBasedRow( + DateTimes.of("2011-01-12T00:00:00.000Z"), + MapVirtualColumnTestBase.mapOf("count", 1L, "params.key3", "value3") + ) + ).stream().map(row -> ResultRow.fromLegacyRow(row, query)).collect(Collectors.toList()); + + Assert.assertEquals(expected, result); + } + + @Test + public void testWithSubColumnWithNotFilter() + { + final GroupByQuery query = new GroupByQuery( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), + VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + NotDimFilter.of(new EqualityFilter("params.key3", ColumnType.STRING, "value3", null)), + Granularities.ALL, + ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), + ImmutableList.of(new CountAggregatorFactory("count")), + null, + null, + null, + null, + null + ); + + final List result = runner.run(QueryPlus.wrap(query)).toList(); + final List expected; + if (NullHandling.sqlCompatible()) { + expected = Collections.emptyList(); + } else { + expected = ImmutableList.of( + new MapBasedRow(DateTimes.of("2011-01-12T00:00:00.000Z"), MapVirtualColumnTestBase.mapOf("count", 2L)) + ).stream().map(row -> ResultRow.fromLegacyRow(row, query)).collect(Collectors.toList()); + } + + Assert.assertEquals(expected, result); + } + + @Test + public void testWithSubColumnWithNotPredicateFilter() + { + final GroupByQuery query = new GroupByQuery( + new TableDataSource(QueryRunnerTestHelper.DATA_SOURCE), + new MultipleIntervalSegmentSpec(ImmutableList.of(Intervals.of("2011/2012"))), + VirtualColumns.create(ImmutableList.of(new MapVirtualColumn("keys", "values", "params"))), + NotDimFilter.of(new InDimFilter("params.key3", ImmutableList.of("value1", "value3"), null)), + Granularities.ALL, + ImmutableList.of(new DefaultDimensionSpec("params.key3", "params.key3")), + ImmutableList.of(new CountAggregatorFactory("count")), + null, + null, + null, + null, + null + ); + + final List result = runner.run(QueryPlus.wrap(query)).toList(); + final List expected; + if (NullHandling.sqlCompatible()) { + expected = Collections.emptyList(); + } else { + expected = ImmutableList.of( + new MapBasedRow(DateTimes.of("2011-01-12T00:00:00.000Z"), MapVirtualColumnTestBase.mapOf("count", 2L)) + ).stream().map(row -> ResultRow.fromLegacyRow(row, query)).collect(Collectors.toList()); + } + + Assert.assertEquals(expected, result); + } } diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java index c2d81cede8c..f85225d107d 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -279,7 +279,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest 10.1, 20.2, Double.NaN, - 10.1, + 2.0, Double.NaN } ); diff --git a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/filter/BloomDimFilter.java b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/filter/BloomDimFilter.java index a22fec72746..159d62b292b 100644 --- a/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/filter/BloomDimFilter.java +++ b/extensions-core/druid-bloom-filter/src/main/java/org/apache/druid/query/filter/BloomDimFilter.java @@ -98,6 +98,8 @@ public class BloomDimFilter extends AbstractOptimizableDimFilter implements DimF dimension, new DruidPredicateFactory() { + private final boolean isNullUnknown = !bloomKFilter.testBytes(null, 0, 0); + @Override public Predicate makeStringPredicate() { @@ -165,6 +167,12 @@ public class BloomDimFilter extends AbstractOptimizableDimFilter implements DimF } }; } + + @Override + public boolean isNullInputUnknown() + { + return isNullUnknown; + } }, extractionFn, filterTuning diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java index 26290da5578..3c59bfc366e 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/filter/BloomDimFilterTest.java @@ -39,6 +39,8 @@ import org.apache.druid.query.lookup.LookupExtractionFn; import org.apache.druid.query.lookup.LookupExtractor; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.filter.BaseFilterTest; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.AfterClass; @@ -69,24 +71,20 @@ public class BloomDimFilterTest extends BaseFilterTest ) ); + private static final RowSignature ROW_SIGNATURE = RowSignature.builder() + .add("dim0", ColumnType.STRING) + .add("dim1", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .add("dim6", ColumnType.STRING) + .build(); + private static final List ROWS = ImmutableList.of( - PARSER.parseBatch(ImmutableMap.of( - "dim0", - "0", - "dim1", - "", - "dim2", - ImmutableList.of("a", "b"), - "dim6", - "2017-07-25" - )).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "1", "dim1", "10", "dim2", ImmutableList.of(), "dim6", "2017-07-25")) - .get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "2", "dim1", "2", "dim2", ImmutableList.of(""), "dim6", "2017-05-25")) - .get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "3", "dim1", "1", "dim2", ImmutableList.of("a"))).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "4", "dim1", "def", "dim2", ImmutableList.of("c"))).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "5", "dim1", "abc")).get(0) + BaseFilterTest.makeSchemaRow(PARSER, ROW_SIGNATURE, "0", "", ImmutableList.of("a", "b"), "2017-07-25"), + BaseFilterTest.makeSchemaRow(PARSER, ROW_SIGNATURE, "1", "10", ImmutableList.of(), "2017-07-25"), + BaseFilterTest.makeSchemaRow(PARSER, ROW_SIGNATURE, "2", "2", ImmutableList.of(""), "2017-05-25"), + BaseFilterTest.makeSchemaRow(PARSER, ROW_SIGNATURE, "3", "1", ImmutableList.of("a")), + BaseFilterTest.makeSchemaRow(PARSER, ROW_SIGNATURE, "4", "def", ImmutableList.of("c")), + BaseFilterTest.makeSchemaRow(PARSER, ROW_SIGNATURE, "5", "abc") ); private static DefaultObjectMapper mapper = new DefaultObjectMapper(); diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java index 239dc980309..4b3854883a2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/common/SortMergeJoinFrameProcessor.java @@ -20,7 +20,6 @@ package org.apache.druid.msq.querykit.common; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import it.unimi.dsi.fastutil.ints.IntOpenHashSet; import it.unimi.dsi.fastutil.ints.IntSet; @@ -47,6 +46,7 @@ import org.apache.druid.msq.indexing.error.TooManyRowsWithSameKeyFault; import org.apache.druid.msq.input.ReadableInput; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelectorFactory; @@ -1041,9 +1041,9 @@ public class SortMergeJoinFrameProcessor implements FrameProcessor } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java index ac247ee8907..fbe82c240db 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/querykit/scan/ExternalColumnSelectorFactory.java @@ -19,10 +19,10 @@ package org.apache.druid.msq.querykit.scan; -import com.google.common.base.Predicate; import org.apache.druid.data.input.InputSource; import org.apache.druid.java.util.common.parsers.ParseException; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelectorFactory; @@ -81,9 +81,9 @@ public class ExternalColumnSelectorFactory implements ColumnSelectorFactory } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return delegateDimensionSelector.makeValueMatcher(predicate); + return delegateDimensionSelector.makeValueMatcher(predicateFactory); } @Override diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java index d771f7497a8..f08311997d9 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQSelectTest.java @@ -793,7 +793,11 @@ public class MSQSelectTest extends MSQTestBase .build()) .setExpectedRowSignature(rowSignature) .setExpectedResultRows( - ImmutableList.of( + NullHandling.sqlCompatible() + ? ImmutableList.of( + new Object[]{"xabc", 1L} + ) + : ImmutableList.of( new Object[]{NullHandling.defaultStringValue(), 3L}, new Object[]{"xabc", 1L} ) diff --git a/processing/src/main/java/org/apache/druid/common/config/NullHandling.java b/processing/src/main/java/org/apache/druid/common/config/NullHandling.java index 43c109c61be..f18242db8fe 100644 --- a/processing/src/main/java/org/apache/druid/common/config/NullHandling.java +++ b/processing/src/main/java/org/apache/druid/common/config/NullHandling.java @@ -22,6 +22,7 @@ package org.apache.druid.common.config; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.inject.Inject; +import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.Indexed; @@ -61,13 +62,19 @@ public class NullHandling @VisibleForTesting public static void initializeForTests() { - INSTANCE = new NullValueHandlingConfig(null, null); + INSTANCE = new NullValueHandlingConfig(null, null, null); } @VisibleForTesting public static void initializeForTestsWithValues(Boolean useDefForNull, Boolean ignoreNullForString) { - INSTANCE = new NullValueHandlingConfig(useDefForNull, ignoreNullForString); + initializeForTestsWithValues(useDefForNull, null, ignoreNullForString); + } + + @VisibleForTesting + public static void initializeForTestsWithValues(Boolean useDefForNull, Boolean useThreeValueLogic, Boolean ignoreNullForString) + { + INSTANCE = new NullValueHandlingConfig(useDefForNull, useThreeValueLogic, ignoreNullForString); } /** @@ -99,6 +106,13 @@ public class NullHandling return !replaceWithDefault(); } + public static boolean useThreeValueLogic() + { + return NullHandling.sqlCompatible() && + INSTANCE.isUseThreeValueLogicForNativeFilters() && + ExpressionProcessing.useStrictBooleans(); + } + @Nullable public static String nullToEmptyIfNeeded(@Nullable String value) { diff --git a/processing/src/main/java/org/apache/druid/common/config/NullValueHandlingConfig.java b/processing/src/main/java/org/apache/druid/common/config/NullValueHandlingConfig.java index fdd13d6a570..2ed36260762 100644 --- a/processing/src/main/java/org/apache/druid/common/config/NullValueHandlingConfig.java +++ b/processing/src/main/java/org/apache/druid/common/config/NullValueHandlingConfig.java @@ -21,10 +21,14 @@ package org.apache.druid.common.config; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; public class NullValueHandlingConfig { + private static final Logger LOG = new Logger(NullValueHandlingConfig.class); public static final String NULL_HANDLING_CONFIG_STRING = "druid.generic.useDefaultValueForNull"; + public static final String THREE_VALUE_LOGIC_CONFIG_STRING = "druid.generic.useThreeValueLogicForNativeFilters"; //added to preserve backward compatibility //and not count nulls during cardinality aggrgation over strings @@ -34,13 +38,16 @@ public class NullValueHandlingConfig @JsonProperty("useDefaultValueForNull") private final boolean useDefaultValuesForNull; + @JsonProperty("useThreeValueLogicForNativeFilters") + private final boolean useThreeValueLogicForNativeFilters; + @JsonProperty("ignoreNullsForStringCardinality") private final boolean ignoreNullsForStringCardinality; - @JsonCreator public NullValueHandlingConfig( @JsonProperty("useDefaultValueForNull") Boolean useDefaultValuesForNull, + @JsonProperty("useThreeValueLogicForNativeFilters") Boolean useThreeValueLogicForNativeFilters, @JsonProperty("ignoreNullsForStringCardinality") Boolean ignoreNullsForStringCardinality ) { @@ -49,6 +56,13 @@ public class NullValueHandlingConfig } else { this.useDefaultValuesForNull = useDefaultValuesForNull; } + if (useThreeValueLogicForNativeFilters == null) { + this.useThreeValueLogicForNativeFilters = Boolean.valueOf( + System.getProperty(THREE_VALUE_LOGIC_CONFIG_STRING, "true") + ); + } else { + this.useThreeValueLogicForNativeFilters = useThreeValueLogicForNativeFilters; + } if (ignoreNullsForStringCardinality == null) { this.ignoreNullsForStringCardinality = Boolean.valueOf(System.getProperty( NULL_HANDLING_DURING_STRING_CARDINALITY, @@ -61,6 +75,24 @@ public class NullValueHandlingConfig this.ignoreNullsForStringCardinality = false; } } + String version = NullValueHandlingConfig.class.getPackage().getImplementationVersion(); + if (version == null || version.contains("SNAPSHOT")) { + version = "latest"; + } + final String docsBaseFormat = "https://druid.apache.org/docs/%s/querying/sql-data-types#%s"; + + if (this.useDefaultValuesForNull) { + LOG.warn( + "druid.generic.useDefaultValueForNull set to 'true', we recommend using 'false' if using SQL to query Druid for the most SQL compliant behavior, see %s for details", + StringUtils.format(docsBaseFormat, version, "null-values") + ); + } + if (!this.useThreeValueLogicForNativeFilters) { + LOG.warn( + "druid.generic.useThreeValueLogic set to 'false', we recommend using 'true' if using SQL to query Druid for the most SQL compliant behavior, see %s for details", + StringUtils.format(docsBaseFormat, version, "boolean-logic") + ); + } } public boolean isIgnoreNullsForStringCardinality() @@ -72,4 +104,9 @@ public class NullValueHandlingConfig { return useDefaultValuesForNull; } + + public boolean isUseThreeValueLogicForNativeFilters() + { + return useThreeValueLogicForNativeFilters; + } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java index 6b15804d662..c0085b46f74 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java @@ -19,7 +19,6 @@ package org.apache.druid.frame.field; -import com.google.common.base.Predicate; import com.google.common.primitives.Ints; import it.unimi.dsi.fastutil.objects.ObjectArrays; import org.apache.datasketches.memory.Memory; @@ -28,6 +27,7 @@ import org.apache.druid.frame.read.FrameReaderUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; @@ -261,9 +261,9 @@ public class StringFieldReader implements FieldReader } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/frame/processor/MultiColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/frame/processor/MultiColumnSelectorFactory.java index 7121e205feb..341cbc31907 100644 --- a/processing/src/main/java/org/apache/druid/frame/processor/MultiColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/processor/MultiColumnSelectorFactory.java @@ -19,9 +19,9 @@ package org.apache.druid.frame.processor; -import com.google.common.base.Predicate; import org.apache.druid.frame.segment.row.FrameColumnSelectorFactory; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelectorFactory; @@ -103,9 +103,9 @@ public class MultiColumnSelectorFactory implements ColumnSelectorFactory } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Nullable diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java index ef29bc1551c..119dd48a3f1 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/StringFrameColumnReader.java @@ -20,7 +20,6 @@ package org.apache.druid.frame.read.columnar; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Predicate; import com.google.common.primitives.Ints; import it.unimi.dsi.fastutil.objects.ObjectArrays; import org.apache.datasketches.memory.Memory; @@ -34,6 +33,7 @@ import org.apache.druid.frame.write.columnar.StringFrameColumnWriter; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.query.rowsandcols.column.Column; @@ -617,9 +617,9 @@ public class StringFrameColumnReader implements FrameColumnReader } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Nullable diff --git a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java index 6dc8abe0c40..3db294801c0 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/FrameFilteredOffset.java @@ -56,7 +56,7 @@ public class FrameFilteredOffset extends SimpleSettableOffset { while (!Thread.currentThread().isInterrupted()) { baseOffset.increment(); - if (!baseOffset.withinBounds() || filterMatcher.matches()) { + if (!baseOffset.withinBounds() || filterMatcher.matches(false)) { return; } } @@ -74,7 +74,7 @@ public class FrameFilteredOffset extends SimpleSettableOffset final int oldOffset = baseOffset.getOffset(); baseOffset.setCurrentOffset(currentOffset); - if (baseOffset.withinBounds() && !filterMatcher.matches()) { + if (baseOffset.withinBounds() && !filterMatcher.matches(false)) { // Offset does not match filter. Invalid; reset to old position and throw an error. baseOffset.setCurrentOffset(oldOffset); throw new ISE("Invalid offset"); @@ -91,7 +91,7 @@ public class FrameFilteredOffset extends SimpleSettableOffset private void incrementIfNeededOnCreationOrReset() { if (baseOffset.withinBounds()) { - if (!filterMatcher.matches()) { + if (!filterMatcher.matches(false)) { increment(); // increment() returns early if it detects the current Thread is interrupted. It will leave this // FilteredOffset in an illegal state, because it may point to an offset that should be filtered. So must diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java index 2f12bd36d7a..daae90ef534 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionProcessingConfig.java @@ -21,11 +21,15 @@ package org.apache.druid.math.expr; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; import javax.annotation.Nullable; public class ExpressionProcessingConfig { + private static final Logger LOG = new Logger(ExpressionProcessingConfig.class); + public static final String NULL_HANDLING_LEGACY_LOGICAL_OPS_STRING = "druid.expressions.useStrictBooleans"; // Coerce arrays to multi value strings public static final String PROCESS_ARRAYS_AS_MULTIVALUE_STRINGS_CONFIG_STRING = @@ -63,6 +67,17 @@ public class ExpressionProcessingConfig homogenizeNullMultiValueStringArrays, HOMOGENIZE_NULL_MULTIVALUE_STRING_ARRAYS ); + String version = ExpressionProcessingConfig.class.getPackage().getImplementationVersion(); + if (version == null || version.contains("SNAPSHOT")) { + version = "latest"; + } + final String docsBaseFormat = "https://druid.apache.org/docs/%s/querying/sql-data-types#%s"; + if (!this.useStrictBooleans) { + LOG.warn( + "druid.expressions.useStrictBooleans set to 'false', we recommend using 'true' if using SQL to query Druid for the most SQL compliant behavior, see %s for details", + StringUtils.format(docsBaseFormat, version, "boolean-logic") + ); + } } public boolean isUseStrictBooleans() diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java index edb6b8a6017..82daa59ab1a 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredAggregator.java @@ -35,7 +35,7 @@ public class FilteredAggregator implements Aggregator @Override public void aggregate() { - if (matcher.matches()) { + if (matcher.matches(false)) { delegate.aggregate(); } } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredBufferAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredBufferAggregator.java index 47e8498189e..0bfa191bce9 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredBufferAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredBufferAggregator.java @@ -44,7 +44,7 @@ public class FilteredBufferAggregator implements BufferAggregator @Override public void aggregate(ByteBuffer buf, int position) { - if (matcher.matches()) { + if (matcher.matches(false)) { delegate.aggregate(buf, position); } } diff --git a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredVectorAggregator.java b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredVectorAggregator.java index 40c0490dbb7..cd055af8247 100644 --- a/processing/src/main/java/org/apache/druid/query/aggregation/FilteredVectorAggregator.java +++ b/processing/src/main/java/org/apache/druid/query/aggregation/FilteredVectorAggregator.java @@ -75,7 +75,7 @@ public class FilteredVectorAggregator implements VectorAggregator mask = maskScratch; } - final ReadableVectorMatch match = matcher.match(mask); + final ReadableVectorMatch match = matcher.match(mask, false); if (match.isAllTrue(matcher.getCurrentVectorSize())) { delegate.aggregate(buf, position, startRow, endRow); @@ -102,7 +102,7 @@ public class FilteredVectorAggregator implements VectorAggregator match0 = VectorMatch.wrap(rows).setSelectionSize(numRows); } - final ReadableVectorMatch match = matcher.match(match0); + final ReadableVectorMatch match = matcher.match(match0, false); final int[] selection = match.getSelection(); if (rows == null) { diff --git a/processing/src/main/java/org/apache/druid/query/dimension/ForwardingFilteredDimensionSelector.java b/processing/src/main/java/org/apache/druid/query/dimension/ForwardingFilteredDimensionSelector.java index 130ddb5a9c1..d9ed240cd02 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/ForwardingFilteredDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/ForwardingFilteredDimensionSelector.java @@ -21,8 +21,10 @@ package org.apache.druid.query.dimension; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; -import com.google.common.base.Predicates; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; @@ -32,7 +34,6 @@ import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.IdMapping; import org.apache.druid.segment.data.ArrayBasedIndexedInts; import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.filter.BooleanValueMatcher; import javax.annotation.Nullable; import java.util.BitSet; @@ -85,60 +86,67 @@ final class ForwardingFilteredDimensionSelector extends AbstractDimensionSelecto IdLookup idLookup = idLookup(); if (idLookup != null) { final int valueId = idLookup.lookupId(value); - if (valueId >= 0 || value == null) { - return new ValueMatcher() + final int nullId = baseIdLookup.lookupId(null); + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) { - @Override - public boolean matches() - { - final IndexedInts baseRow = selector.getRow(); - final int baseRowSize = baseRow.size(); - boolean nullRow = true; - for (int i = 0; i < baseRowSize; i++) { - int forwardedValue = idMapping.getForwardedId(baseRow.get(i)); - if (forwardedValue >= 0) { - // Make the following check after the `forwardedValue >= 0` check, because if forwardedValue is -1 and - // valueId is -1, we don't want to return true from matches(). - if (forwardedValue == valueId) { - return true; - } - nullRow = false; - } + final IndexedInts baseRow = selector.getRow(); + final int baseRowSize = baseRow.size(); + boolean nullRow = true; + for (int i = 0; i < baseRowSize; i++) { + final int baseId = baseRow.get(i); + if (includeUnknown && nullId == baseId) { + return true; + } + final int forwardedId = idMapping.getForwardedId(baseId); + if (forwardedId >= 0) { + // Make the following check after the `forwardedId >= 0` check, because if forwardedId is -1 and + // valueId is -1, we don't want to return true from matches(). + if (forwardedId == valueId) { + return true; + } + nullRow = false; } - // null should match empty rows in multi-value columns - return nullRow && value == null; } + // null should match empty rows in multi-value columns + return nullRow && (includeUnknown || value == null); + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("selector", selector); - } - }; - } else { - return BooleanValueMatcher.of(false); - } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; } else { // Employ precomputed BitSet optimization - return makeValueMatcher(Predicates.equalTo(value)); + return makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(value)); } } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { + final Predicate predicate = predicateFactory.makeStringPredicate(); final BitSet valueIds = DimensionSelectorUtils.makePredicateMatchingSet(this, predicate); - final boolean matchNull = predicate.apply(null); + final boolean predicateMatchesNull = predicate.apply(null); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final IndexedInts baseRow = selector.getRow(); final int baseRowSize = baseRow.size(); boolean nullRow = true; for (int i = 0; i < baseRowSize; ++i) { - int forwardedValue = idMapping.getForwardedId(baseRow.get(i)); + final int baseId = baseRow.get(i); + + if (includeUnknown && NullHandling.isNullOrEquivalent(selector.lookupName(baseId))) { + return true; + } + int forwardedValue = idMapping.getForwardedId(baseId); if (forwardedValue >= 0) { if (valueIds.get(forwardedValue)) { return true; @@ -147,7 +155,7 @@ final class ForwardingFilteredDimensionSelector extends AbstractDimensionSelecto } } // null should match empty rows in multi-value columns - return nullRow && matchNull; + return nullRow && (includeUnknown || predicateMatchesNull); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/dimension/PredicateFilteredDimensionSelector.java b/processing/src/main/java/org/apache/druid/query/dimension/PredicateFilteredDimensionSelector.java index cf4e88bbe88..203f5adb1fb 100644 --- a/processing/src/main/java/org/apache/druid/query/dimension/PredicateFilteredDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/query/dimension/PredicateFilteredDimensionSelector.java @@ -20,6 +20,7 @@ package org.apache.druid.query.dimension; import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; @@ -67,13 +68,16 @@ final class PredicateFilteredDimensionSelector extends AbstractDimensionSelector return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final IndexedInts baseRow = selector.getRow(); final int baseRowSize = baseRow.size(); boolean nullRow = true; for (int i = 0; i < baseRowSize; i++) { String rowValue = lookupName(baseRow.get(i)); + if (includeUnknown && rowValue == null) { + return true; + } if (predicate.apply(rowValue)) { if (Objects.equals(rowValue, value)) { return true; @@ -82,7 +86,7 @@ final class PredicateFilteredDimensionSelector extends AbstractDimensionSelector } } // null should match empty rows in multi-value columns - return nullRow && value == null; + return nullRow && (includeUnknown || value == null); } @Override @@ -95,19 +99,24 @@ final class PredicateFilteredDimensionSelector extends AbstractDimensionSelector } @Override - public ValueMatcher makeValueMatcher(final Predicate matcherPredicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { - final boolean matchNull = matcherPredicate.apply(null); + final Predicate matcherPredicate = predicateFactory.makeStringPredicate(); + final boolean predicateMatchesNull = matcherPredicate.apply(null); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); final IndexedInts baseRow = selector.getRow(); final int baseRowSize = baseRow.size(); boolean nullRow = true; for (int i = 0; i < baseRowSize; ++i) { String rowValue = lookupName(baseRow.get(i)); + if (matchNull && rowValue == null) { + return true; + } if (predicate.apply(rowValue)) { if (matcherPredicate.apply(rowValue)) { return true; @@ -116,7 +125,7 @@ final class PredicateFilteredDimensionSelector extends AbstractDimensionSelector } } // null should match empty rows in multi-value columns - return nullRow && matchNull; + return nullRow && (includeUnknown || predicateMatchesNull); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java b/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java index 929f3acbba5..19a7bf24abb 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DruidPredicateFactory.java @@ -22,6 +22,8 @@ package org.apache.druid.query.filter; import com.google.common.base.Predicate; import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode; import org.apache.druid.java.util.common.UOE; +import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.query.filter.vector.ReadableVectorMatch; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.ValueType; @@ -58,4 +60,19 @@ public interface DruidPredicateFactory final Predicate stringPredicate = makeStringPredicate(); return o -> stringPredicate.apply(null); } + + /** + * Indicator for if null inputs should be considered 'unknown' matches when used for filter matching with + * {@link ValueMatcher#matches(boolean)}, + * {@link org.apache.druid.query.filter.vector.VectorValueMatcher#match(ReadableVectorMatch, boolean)}, or + * {@link org.apache.druid.segment.index.BitmapColumnIndex#computeBitmapResult(BitmapResultFactory, boolean)}. + * + * If returns true, unknown (null) inputs can automatically be considered matches if {@code includeUnknown} is set + * to true on these methods, else null inputs should be evaluated against the predicate as any other value to + * determine a match + */ + default boolean isNullInputUnknown() + { + return true; + } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java b/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java index ef507e4f20b..636f95fecf5 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/EqualityFilter.java @@ -53,11 +53,10 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.TypeSignature; import org.apache.druid.segment.column.TypeStrategy; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.filter.BooleanValueMatcher; import org.apache.druid.segment.filter.Filters; import org.apache.druid.segment.filter.PredicateValueMatcherFactory; import org.apache.druid.segment.filter.ValueMatchers; -import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.index.semantic.StringValueSetIndexes; import org.apache.druid.segment.index.semantic.ValueIndexes; @@ -235,7 +234,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column); if (indexSupplier == null) { - return new AllFalseBitmapColumnIndex(selector); + return new AllUnknownBitmapColumnIndex(selector); } final ValueIndexes valueIndexes = indexSupplier.as(ValueIndexes.class); @@ -538,7 +537,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt { final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.STRING); if (castForComparison == null) { - return BooleanValueMatcher.of(false); + return ValueMatchers.makeAlwaysFalseDimensionMatcher(selector, multiValue); } return ValueMatchers.makeStringValueMatcher(selector, castForComparison.asString(), multiValue); } @@ -548,7 +547,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt { final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE); if (castForComparison == null) { - return BooleanValueMatcher.of(false); + return ValueMatchers.makeAlwaysFalseNumericMatcher(selector); } return ValueMatchers.makeFloatValueMatcher(selector, (float) castForComparison.asDouble()); } @@ -558,7 +557,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt { final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.DOUBLE); if (castForComparison == null) { - return BooleanValueMatcher.of(false); + return ValueMatchers.makeAlwaysFalseNumericMatcher(selector); } return ValueMatchers.makeDoubleValueMatcher(selector, castForComparison.asDouble()); } @@ -568,7 +567,7 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt { final ExprEval castForComparison = ExprEval.castForEqualityComparison(matchValue, ExpressionType.LONG); if (castForComparison == null) { - return BooleanValueMatcher.of(false); + return ValueMatchers.makeAlwaysFalseNumericMatcher(selector); } return ValueMatchers.makeLongValueMatcher(selector, castForComparison.asLong()); } diff --git a/processing/src/main/java/org/apache/druid/query/filter/Filter.java b/processing/src/main/java/org/apache/druid/query/filter/Filter.java index 87d34e2e26b..49ec8240904 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/Filter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/Filter.java @@ -39,8 +39,8 @@ public interface Filter /** * Returns a {@link BitmapColumnIndex} if this filter supports using a bitmap index for filtering for the given input * {@link ColumnIndexSelector}. The {@link BitmapColumnIndex} can be used to compute into a bitmap indicating rows - * that match this filter result {@link BitmapColumnIndex#computeBitmapResult(BitmapResultFactory)}, or examine - * details about the index prior to computing it, via {@link BitmapColumnIndex#getIndexCapabilities()}. + * that match this filter result {@link BitmapColumnIndex#computeBitmapResult(BitmapResultFactory, boolean)}, or + * examine details about the index prior to computing it, via {@link BitmapColumnIndex#getIndexCapabilities()}. * * @param selector Object used to create BitmapColumnIndex * @@ -50,7 +50,7 @@ public interface Filter BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector); /** - * Get a ValueMatcher that applies this filter to row values. + * Get a {@link ValueMatcher} that applies this filter to row values. * * @param factory Object used to create ValueMatchers * @@ -59,7 +59,7 @@ public interface Filter ValueMatcher makeMatcher(ColumnSelectorFactory factory); /** - * Get a VectorValueMatcher that applies this filter to row vectors. + * Get a {@link VectorValueMatcher} that applies this filter to row vectors. * * @param factory Object used to create ValueMatchers * 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 fcbd6aa4960..329e652ce0b 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 @@ -564,6 +564,7 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter private final Supplier longPredicateSupplier; private final Supplier floatPredicateSupplier; private final Supplier doublePredicateSupplier; + private final boolean hasNull; public InFilterDruidPredicateFactory( final ExtractionFn extractionFn, @@ -572,6 +573,7 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter { this.extractionFn = extractionFn; this.values = values; + this.hasNull = values.contains(null); // 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 @@ -627,6 +629,12 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter } } + @Override + public boolean isNullInputUnknown() + { + return !hasNull; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java index 0b154694381..4de90811a77 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/NotDimFilter.java @@ -34,6 +34,11 @@ import java.util.Set; */ public class NotDimFilter extends AbstractOptimizableDimFilter implements DimFilter { + public static NotDimFilter of(DimFilter field) + { + return new NotDimFilter(field); + } + private final DimFilter field; diff --git a/processing/src/main/java/org/apache/druid/query/filter/NullFilter.java b/processing/src/main/java/org/apache/druid/query/filter/NullFilter.java index 1f504e62d2c..14bdb6f4185 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/NullFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/NullFilter.java @@ -267,6 +267,13 @@ public class NullFilter extends AbstractOptimizableDimFilter implements Filter return Predicates.isNull(); } + @Override + public boolean isNullInputUnknown() + { + // this filter only matches null inputs + return false; + } + @Override public int hashCode() { diff --git a/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java b/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java index f65bb6870cc..1e7f21e17c6 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/RangeFilter.java @@ -52,7 +52,7 @@ import org.apache.druid.segment.column.TypeStrategy; import org.apache.druid.segment.column.Types; import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; +import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; @@ -313,7 +313,7 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter } final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(column); if (indexSupplier == null) { - return new AllFalseBitmapColumnIndex(selector); + return new AllUnknownBitmapColumnIndex(selector); } if (matchValueType.is(ValueType.STRING)) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/SelectorPredicateFactory.java b/processing/src/main/java/org/apache/druid/query/filter/SelectorPredicateFactory.java index 7262d9811c8..538b26474ab 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/SelectorPredicateFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/SelectorPredicateFactory.java @@ -40,10 +40,12 @@ public class SelectorPredicateFactory implements DruidPredicateFactory private volatile DruidLongPredicate longPredicate; private volatile DruidFloatPredicate floatPredicate; private volatile DruidDoublePredicate doublePredicate; + private final boolean isNullUnknown; public SelectorPredicateFactory(@Nullable String value) { this.value = value; + this.isNullUnknown = value != null; } @Override @@ -73,6 +75,12 @@ public class SelectorPredicateFactory implements DruidPredicateFactory return doublePredicate; } + @Override + public boolean isNullInputUnknown() + { + return isNullUnknown; + } + private void initLongPredicate() { if (longPredicate != null) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/StringPredicateDruidPredicateFactory.java b/processing/src/main/java/org/apache/druid/query/filter/StringPredicateDruidPredicateFactory.java new file mode 100644 index 00000000000..a043e0d6962 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/StringPredicateDruidPredicateFactory.java @@ -0,0 +1,99 @@ +/* + * 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.google.common.base.Predicate; +import com.google.common.base.Predicates; +import org.apache.druid.error.DruidException; + +import javax.annotation.Nullable; +import java.util.Objects; + +public class StringPredicateDruidPredicateFactory implements DruidPredicateFactory +{ + public static StringPredicateDruidPredicateFactory equalTo(@Nullable String value) + { + return new StringPredicateDruidPredicateFactory(Predicates.equalTo(value), value != null); + } + + public static StringPredicateDruidPredicateFactory of(@Nullable Predicate predicate, boolean isNullInputUnknown) + { + return new StringPredicateDruidPredicateFactory(predicate, isNullInputUnknown); + } + + private final boolean isNullInputUnknown; + @Nullable + private final Predicate predicate; + + public StringPredicateDruidPredicateFactory(Predicate predicate, boolean isNullInputUnknown) + { + this.predicate = predicate; + this.isNullInputUnknown = isNullInputUnknown; + } + + @Override + public Predicate makeStringPredicate() + { + return predicate; + } + + @Override + public DruidLongPredicate makeLongPredicate() + { + throw DruidException.defensive("String equality predicate factory only supports string predicates"); + } + + @Override + public DruidFloatPredicate makeFloatPredicate() + { + throw DruidException.defensive("String equality predicate factory only supports string predicates"); + } + + @Override + public DruidDoublePredicate makeDoublePredicate() + { + throw DruidException.defensive("String equality predicate factory only supports string predicates"); + } + + @Override + public boolean isNullInputUnknown() + { + return isNullInputUnknown; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StringPredicateDruidPredicateFactory that = (StringPredicateDruidPredicateFactory) o; + return isNullInputUnknown == that.isNullInputUnknown && Objects.equals(predicate, that.predicate); + } + + @Override + public int hashCode() + { + return Objects.hash(isNullInputUnknown, predicate); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java index c5f73886b7e..168b9d3860b 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/ValueMatcher.java @@ -31,6 +31,16 @@ import org.apache.druid.query.monomorphicprocessing.HotLoopCallee; */ public interface ValueMatcher extends HotLoopCallee { + /** + * Returns true if the current row matches the condition. + * + * @param includeUnknown mapping for Druid native two state logic system into SQL three-state logic system. If set + * to true, this method should also return true if the result is 'unknown' to be a match, such + * as from the input being null valued. Used primarily to allow + * {@link org.apache.druid.segment.filter.NotFilter} to invert a match in a SQL compliant + * manner + * @return true if the current row matches the condition, or is unknown and {@code includeUnknown} is set to true + */ @CalledFromHotLoop - boolean matches(); + boolean matches(boolean includeUnknown); } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java index 7d889c85aa8..2a43d6aa755 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/ArrayVectorValueMatcher.java @@ -62,13 +62,12 @@ public class ArrayVectorValueMatcher implements VectorValueMatcherFactory public VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory) { final Predicate predicate = predicateFactory.makeArrayPredicate(columnType); - return new BaseVectorValueMatcher(selector) { final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final Object[] vector = selector.getObjectVector(); final int[] selection = match.getSelection(); @@ -78,7 +77,9 @@ public class ArrayVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; Object o = vector[rowNum]; - if (o == null || o instanceof Object[]) { + if (includeUnknown && o == null && predicateFactory.isNullInputUnknown()) { + selection[numRows++] = rowNum; + } else if (o == null || o instanceof Object[]) { if (predicate.apply((Object[]) o)) { selection[numRows++] = rowNum; } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java index efa0236acfd..2da07cf6af5 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/BooleanVectorValueMatcher.java @@ -19,27 +19,28 @@ package org.apache.druid.query.filter.vector; +import org.apache.druid.segment.filter.ConstantMatcherType; import org.apache.druid.segment.vector.VectorSizeInspector; -public class BooleanVectorValueMatcher extends BaseVectorValueMatcher +public final class BooleanVectorValueMatcher extends BaseVectorValueMatcher { - private final boolean matches; + private final ConstantMatcherType type; - private BooleanVectorValueMatcher(final VectorSizeInspector selector, final boolean matches) + private BooleanVectorValueMatcher(final VectorSizeInspector selector, final ConstantMatcherType type) { super(selector); - this.matches = matches; + this.type = type; } - public static BooleanVectorValueMatcher of(final VectorSizeInspector selector, final boolean matches) + public static BooleanVectorValueMatcher of(final VectorSizeInspector selector, ConstantMatcherType matcherType) { - return new BooleanVectorValueMatcher(selector, matches); + return new BooleanVectorValueMatcher(selector, matcherType); } @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { - if (matches) { + if (type == ConstantMatcherType.ALL_TRUE || (includeUnknown && type == ConstantMatcherType.ALL_UNKNOWN)) { assert mask.isValid(mask); return mask; } else { diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java index 91249ae5612..fce119ef467 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/DoubleVectorValueMatcher.java @@ -42,13 +42,14 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory public VectorValueMatcher makeMatcher(@Nullable final String value) { if (value == null) { - return makeNullValueMatcher(selector); + // special case for selector filter, which behaves as both '=' and 'is null' + return VectorValueMatcher.nullMatcher(selector); } final Double matchVal = DimensionHandlerUtils.convertObjectToDouble(value); if (matchVal == null) { - return BooleanVectorValueMatcher.of(selector, false); + return VectorValueMatcher.allFalseValueMatcher(selector); } return makeDoubleMatcher(matchVal); @@ -59,11 +60,8 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory { final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE); - if (castForComparison == null) { - return BooleanVectorValueMatcher.of(selector, false); - } - if (castForComparison.isNumericNull()) { - return makeNullValueMatcher(selector); + if (castForComparison == null || castForComparison.isNumericNull()) { + return VectorValueMatcher.allFalseValueMatcher(selector); } return makeDoubleMatcher(castForComparison.asDouble()); } @@ -75,7 +73,7 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final double[] vector = selector.getDoubleVector(); final int[] selection = match.getSelection(); @@ -86,9 +84,10 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; if (hasNulls && nulls[rowNum]) { - continue; - } - if (vector[rowNum] == matchValDouble) { + if (includeUnknown) { + selection[numRows++] = rowNum; + } + } else if (vector[rowNum] == matchValDouble) { selection[numRows++] = rowNum; } } @@ -110,7 +109,7 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final double[] vector = selector.getDoubleVector(); final int[] selection = match.getSelection(); @@ -122,7 +121,7 @@ public class DoubleVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; if (hasNulls && nulls[rowNum]) { - if (predicate.applyNull()) { + if ((includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.applyNull()) { selection[numRows++] = rowNum; } } else if (predicate.applyDouble(vector[rowNum])) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java index 823f88d3cf2..e86abd863c3 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/FloatVectorValueMatcher.java @@ -42,13 +42,14 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory public VectorValueMatcher makeMatcher(@Nullable final String value) { if (value == null) { - return makeNullValueMatcher(selector); + // special case for selector filter, which is both '=' and 'is null' + return VectorValueMatcher.nullMatcher(selector); } final Float matchVal = DimensionHandlerUtils.convertObjectToFloat(value); if (matchVal == null) { - return BooleanVectorValueMatcher.of(selector, false); + return VectorValueMatcher.allFalseValueMatcher(selector); } final float matchValFloat = matchVal; @@ -61,11 +62,8 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory { final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE); - if (castForComparison == null) { - return BooleanVectorValueMatcher.of(selector, false); - } - if (castForComparison.isNumericNull()) { - return makeNullValueMatcher(selector); + if (castForComparison == null || castForComparison.isNumericNull()) { + return VectorValueMatcher.allFalseValueMatcher(selector); } return makeFloatMatcher((float) castForComparison.asDouble()); } @@ -77,7 +75,7 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final float[] vector = selector.getFloatVector(); final int[] selection = match.getSelection(); @@ -89,9 +87,10 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; if (hasNulls && nulls[rowNum]) { - continue; - } - if (vector[rowNum] == matchValFloat) { + if (includeUnknown) { + selection[numRows++] = rowNum; + } + } else if (vector[rowNum] == matchValFloat) { selection[numRows++] = rowNum; } } @@ -112,7 +111,7 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final float[] vector = selector.getFloatVector(); final int[] selection = match.getSelection(); @@ -124,7 +123,7 @@ public class FloatVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; if (hasNulls && nulls[rowNum]) { - if (predicate.applyNull()) { + if ((includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.applyNull()) { selection[numRows++] = rowNum; } } else if (predicate.applyFloat(vector[rowNum])) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java index 7eb865f7aa8..316564dd49d 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/LongVectorValueMatcher.java @@ -42,13 +42,13 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory public VectorValueMatcher makeMatcher(@Nullable final String value) { if (value == null) { - return makeNullValueMatcher(selector); + return VectorValueMatcher.nullMatcher(selector); } final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value); if (matchVal == null) { - return BooleanVectorValueMatcher.of(selector, false); + return VectorValueMatcher.allFalseValueMatcher(selector); } final long matchValLong = matchVal; @@ -61,11 +61,8 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory { final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.LONG); - if (castForComparison == null) { - return BooleanVectorValueMatcher.of(selector, false); - } - if (castForComparison.isNumericNull()) { - return makeNullValueMatcher(selector); + if (castForComparison == null || castForComparison.isNumericNull()) { + return VectorValueMatcher.allFalseValueMatcher(selector); } return makeLongMatcher(castForComparison.asLong()); } @@ -77,7 +74,7 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final long[] vector = selector.getLongVector(); final int[] selection = match.getSelection(); @@ -89,9 +86,10 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; if (hasNulls && nulls[rowNum]) { - continue; - } - if (vector[rowNum] == matchValLong) { + if (includeUnknown) { + selection[numRows++] = rowNum; + } + } else if (vector[rowNum] == matchValLong) { selection[numRows++] = rowNum; } } @@ -112,7 +110,7 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final long[] vector = selector.getLongVector(); final int[] selection = match.getSelection(); @@ -124,7 +122,7 @@ public class LongVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; if (hasNulls && nulls[rowNum]) { - if (predicate.applyNull()) { + if ((includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.applyNull()) { selection[numRows++] = rowNum; } } else if (predicate.applyLong(vector[rowNum])) { diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java index 000be2a26a6..c164c2e698e 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/MultiValueStringVectorValueMatcher.java @@ -45,17 +45,18 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac @Override public VectorValueMatcher makeMatcher(@Nullable final String value) { - final String etnValue = NullHandling.emptyToNullIfNeeded(NullHandling.emptyToNullIfNeeded(value)); + final String etnValue = NullHandling.emptyToNullIfNeeded(value); final IdLookup idLookup = selector.idLookup(); final int id; if (idLookup != null) { // Optimization when names can be looked up to IDs ahead of time. id = idLookup.lookupId(etnValue); + final boolean hasNull = NullHandling.isNullOrEquivalent(selector.lookupName(0)); if (id < 0) { // Value doesn't exist in this column. - return BooleanVectorValueMatcher.of(selector, false); + return VectorValueMatcher.allFalseMultiValueDimensionMatcher(selector); } // Check for "id". @@ -64,7 +65,7 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final IndexedInts[] vector = selector.getRowVector(); final int[] selection = match.getSelection(); @@ -78,12 +79,13 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac if (n == 0) { // null should match empty rows in multi-value columns - if (etnValue == null) { + if (etnValue == null || includeUnknown) { selection[numRows++] = rowNum; } } else { for (int j = 0; j < n; j++) { - if (ints.get(j) == id) { + final int rowId = ints.get(j); + if ((hasNull && includeUnknown && rowId == 0) || (rowId == id)) { selection[numRows++] = rowNum; break; } @@ -96,7 +98,7 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac } }; } else { - return makeMatcher(s -> Objects.equals(s, etnValue)); + return makeMatcher(s -> Objects.equals(s, etnValue), true); } } @@ -106,7 +108,7 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING); if (castForComparison == null) { - return BooleanVectorValueMatcher.of(selector, false); + return VectorValueMatcher.allFalseMultiValueDimensionMatcher(selector); } return makeMatcher(castForComparison.asString()); } @@ -114,10 +116,10 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac @Override public VectorValueMatcher makeMatcher(final DruidPredicateFactory predicateFactory) { - return makeMatcher(predicateFactory.makeStringPredicate()); + return makeMatcher(predicateFactory.makeStringPredicate(), predicateFactory.isNullInputUnknown()); } - private VectorValueMatcher makeMatcher(final Predicate predicate) + private VectorValueMatcher makeMatcher(final Predicate predicate, boolean isNullInputUnknown) { final boolean matchNull = predicate.apply(null); @@ -131,10 +133,11 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac private final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final IndexedInts[] vector = selector.getRowVector(); final int[] selection = match.getSelection(); + final boolean includeNulls = includeUnknown && isNullInputUnknown; int numRows = 0; @@ -145,7 +148,7 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac if (n == 0) { // null should match empty rows in multi-value columns - if (matchNull) { + if (matchNull || includeNulls) { selection[numRows++] = rowNum; } } else { @@ -156,7 +159,8 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac if (checkedIds.get(id)) { matches = matchingIds.get(id); } else { - matches = predicate.apply(selector.lookupName(id)); + final String val = selector.lookupName(id); + matches = (includeNulls && val == null) || predicate.apply(val); checkedIds.set(id); if (matches) { matchingIds.set(id); @@ -182,8 +186,9 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { + final boolean includeNulls = includeUnknown && isNullInputUnknown; final IndexedInts[] vector = selector.getRowVector(); final int[] selection = match.getSelection(); @@ -196,13 +201,14 @@ public class MultiValueStringVectorValueMatcher implements VectorValueMatcherFac if (n == 0) { // null should match empty rows in multi-value columns - if (matchNull) { + if (matchNull || includeNulls) { selection[numRows++] = rowNum; } } else { for (int j = 0; j < n; j++) { final int id = ints.get(j); - if (predicate.apply(selector.lookupName(id))) { + final String val = selector.lookupName(id); + if ((includeNulls && val == null) || predicate.apply(val)) { selection[numRows++] = rowNum; break; } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java index c464985be87..eb8ef3fa651 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/ObjectVectorValueMatcher.java @@ -22,6 +22,7 @@ package org.apache.druid.query.filter.vector; import com.google.common.base.Predicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.filter.ConstantMatcherType; import org.apache.druid.segment.vector.VectorObjectSelector; import javax.annotation.Nullable; @@ -50,13 +51,19 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory public VectorValueMatcher makeMatcher(@Nullable String value) { // return a traditional nil matcher, as is the custom of our people - return BooleanVectorValueMatcher.of(selector, value == null); + if (value == null) { + return ConstantMatcherType.ALL_TRUE.asVectorMatcher(selector); + } + return VectorValueMatcher.allFalseObjectMatcher(selector); } @Override public VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType) { - return BooleanVectorValueMatcher.of(selector, matchValue == null); + if (matchValue == null) { + return ConstantMatcherType.ALL_TRUE.asVectorMatcher(selector); + } + return VectorValueMatcher.allFalseObjectMatcher(selector); } @Override @@ -69,7 +76,7 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final Object[] vector = selector.getObjectVector(); final int[] selection = match.getSelection(); @@ -78,7 +85,8 @@ public class ObjectVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; - if (predicate.apply(vector[rowNum])) { + final Object o = vector[rowNum]; + if ((o == null && includeUnknown && predicateFactory.isNullInputUnknown()) || predicate.apply(o)) { selection[numRows++] = rowNum; } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java index 052758c1220..e76f04c55ac 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/SingleValueStringVectorValueMatcher.java @@ -26,6 +26,7 @@ import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.IdLookup; import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.filter.ConstantMatcherType; import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; @@ -42,29 +43,18 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa this.selector = selector; } - @Nullable - private static BooleanVectorValueMatcher toBooleanMatcherIfPossible( - final SingleValueDimensionVectorSelector selector, - final Predicate predicate - ) - { - final Boolean booleanValue = ValueMatchers.toBooleanIfPossible( - selector, - false, - predicate - ); - - return booleanValue == null ? null : BooleanVectorValueMatcher.of(selector, booleanValue); - } - @Override public VectorValueMatcher makeMatcher(@Nullable final String value) { final String etnValue = NullHandling.emptyToNullIfNeeded(value); - final VectorValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, s -> Objects.equals(s, etnValue)); - if (booleanMatcher != null) { - return booleanMatcher; + final ConstantMatcherType constantMatcherType = ValueMatchers.toConstantMatcherTypeIfPossible( + selector, + false, + s -> Objects.equals(s, etnValue) + ); + if (constantMatcherType != null) { + return constantMatcherType.asVectorMatcher(selector); } final IdLookup idLookup = selector.idLookup(); @@ -76,8 +66,9 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa if (id < 0) { // Value doesn't exist in this column. - return BooleanVectorValueMatcher.of(selector, false); + return VectorValueMatcher.allFalseSingleValueDimensionMatcher(selector); } + final boolean hasNull = NullHandling.isNullOrEquivalent(selector.lookupName(0)); // Check for "id". return new BaseVectorValueMatcher(selector) @@ -85,7 +76,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final int[] vector = selector.getRowVector(); final int[] selection = match.getSelection(); @@ -94,7 +85,8 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; - if (vector[rowNum] == id) { + final int rowId = vector[rowNum]; + if ((includeUnknown && hasNull && rowId == 0) || rowId == id) { selection[numRows++] = rowNum; } } @@ -113,8 +105,8 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa { final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING); - if (castForComparison == null) { - return BooleanVectorValueMatcher.of(selector, false); + if (castForComparison == null || castForComparison.asString() == null) { + return VectorValueMatcher.allFalseSingleValueDimensionMatcher(selector); } return makeMatcher(castForComparison.asString()); } @@ -127,9 +119,14 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa private VectorValueMatcher makeMatcher(final Predicate predicate) { - final VectorValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, predicate); - if (booleanMatcher != null) { - return booleanMatcher; + final ConstantMatcherType constantMatcherType = ValueMatchers.toConstantMatcherTypeIfPossible( + selector, + false, + predicate + ); + + if (constantMatcherType != null) { + return constantMatcherType.asVectorMatcher(selector); } if (selector.getValueCardinality() > 0) { @@ -142,7 +139,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa private final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final int[] vector = selector.getRowVector(); final int[] selection = match.getSelection(); @@ -157,7 +154,8 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa if (checkedIds.get(id)) { matches = matchingIds.get(id); } else { - matches = predicate.apply(selector.lookupName(id)); + final String val = selector.lookupName(id); + matches = (includeUnknown && val == null) || predicate.apply(val); checkedIds.set(id); if (matches) { matchingIds.set(id); @@ -180,7 +178,7 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final int[] vector = selector.getRowVector(); final int[] selection = match.getSelection(); @@ -189,7 +187,8 @@ public class SingleValueStringVectorValueMatcher implements VectorValueMatcherFa for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; - if (predicate.apply(selector.lookupName(vector[rowNum]))) { + final String val = selector.lookupName(vector[rowNum]); + if ((includeUnknown && val == null) || predicate.apply(val)) { selection[numRows++] = rowNum; } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java index 4078394ccfe..051cbbdb57c 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/StringObjectVectorValueMatcher.java @@ -49,7 +49,7 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final Object[] vector = selector.getObjectVector(); final int[] selection = match.getSelection(); @@ -58,7 +58,7 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; - if (Objects.equals(value, vector[rowNum])) { + if ((value == null && includeUnknown) || Objects.equals(value, vector[rowNum])) { selection[numRows++] = rowNum; } } @@ -74,8 +74,8 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory { final ExprEval eval = ExprEval.ofType(ExpressionType.fromColumnType(matchValueType), matchValue); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.STRING); - if (castForComparison == null) { - return BooleanVectorValueMatcher.of(selector, false); + if (castForComparison == null || castForComparison.asString() == null) { + return VectorValueMatcher.allFalseObjectMatcher(selector); } return makeMatcher(castForComparison.asString()); } @@ -90,7 +90,7 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { final Object[] vector = selector.getObjectVector(); final int[] selection = match.getSelection(); @@ -99,7 +99,8 @@ public class StringObjectVectorValueMatcher implements VectorValueMatcherFactory for (int i = 0; i < mask.getSelectionSize(); i++) { final int rowNum = mask.getSelection()[i]; - if (predicate.apply((String) vector[rowNum])) { + final String val = (String) vector[rowNum]; + if ((includeUnknown && val == null) || predicate.apply(val)) { selection[numRows++] = rowNum; } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcher.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcher.java index bb0a44a2c0a..b9208d5b3ce 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcher.java @@ -19,8 +19,18 @@ package org.apache.druid.query.filter.vector; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.filter.Filter; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.segment.filter.ConstantMatcherType; +import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector; +import org.apache.druid.segment.vector.SingleValueDimensionVectorSelector; +import org.apache.druid.segment.vector.VectorObjectSelector; import org.apache.druid.segment.vector.VectorSizeInspector; +import org.apache.druid.segment.vector.VectorValueSelector; + +import javax.annotation.Nullable; /** * An object that returns a boolean indicating if the "current" row should be selected or not. The most prominent use @@ -33,13 +43,279 @@ public interface VectorValueMatcher extends VectorSizeInspector { /** * Examine the current vector and return a match indicating what is accepted. - * + *

* Does not modify "mask". * - * @param mask must not be null; use {@link VectorMatch#allTrue} if you don't need a mask. - * + * @param mask must not be null; use {@link VectorMatch#allTrue} if you don't need a mask. + * @param includeUnknown mapping for Druid native two state logic system into SQL three-state logic system. If set + * to true, this method should also return true if the result is 'unknown' to be a match, such + * as from the input being null valued. Used primarily to allow + * {@link org.apache.druid.segment.filter.NotFilter} to invert a match in an SQL compliant + * manner * @return the subset of "mask" that this value matcher accepts. May be the same instance as {@param mask} if * every row in the mask matches the filter. */ - ReadableVectorMatch match(ReadableVectorMatch mask); + ReadableVectorMatch match(ReadableVectorMatch mask, boolean includeUnknown); + + /** + * Make a {@link VectorValueMatcher} that only selects input rows with null values + * @param selector + * @return + */ + static VectorValueMatcher nullMatcher(VectorValueSelector selector) + { + return new BaseVectorValueMatcher(selector) + { + final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); + + @Override + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) + { + return matchNulls(mask, match, selector.getNullVector()); + } + }; + } + + /** + * Make an always false {@link VectorValueMatcher} for a {@link SingleValueDimensionVectorSelector}. When + * {@code includeUnknown} is specified to the {@link VectorValueMatcher#match(ReadableVectorMatch, boolean)} function, + * this matcher will add all rows of {@link SingleValueDimensionVectorSelector#getRowVector()} which are null to the + * {@link ReadableVectorMatch} as selections, to participate in Druid 2-state logic system to SQL 3-state logic + * system conversion. + */ + static VectorValueMatcher allFalseSingleValueDimensionMatcher(SingleValueDimensionVectorSelector selector) + { + final IdLookup idLookup = selector.idLookup(); + final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); + + if (idLookup == null || !selector.nameLookupPossibleInAdvance()) { + // must call selector.lookupName on every id to check for nulls + return new BaseVectorValueMatcher(selector) + { + @Override + public ReadableVectorMatch match(ReadableVectorMatch mask, boolean includeUnknown) + { + if (includeUnknown) { + final int[] vector = selector.getRowVector(); + final int[] inputSelection = mask.getSelection(); + final int inputSelectionSize = mask.getSelectionSize(); + final int[] outputSelection = match.getSelection(); + int outputSelectionSize = 0; + + for (int i = 0; i < inputSelectionSize; i++) { + final int rowNum = inputSelection[i]; + if (NullHandling.isNullOrEquivalent(selector.lookupName(vector[rowNum]))) { + outputSelection[outputSelectionSize++] = rowNum; + } + } + match.setSelectionSize(outputSelectionSize); + return match; + } + return VectorMatch.allFalse(); + } + }; + } else { + final int nullId = idLookup.lookupId(null); + // column doesn't have nulls, can safely return an 'all false' matcher + if (nullId < 0) { + return ConstantMatcherType.ALL_FALSE.asVectorMatcher(selector); + } + + return new BaseVectorValueMatcher(selector) + { + @Override + public ReadableVectorMatch match(ReadableVectorMatch mask, boolean includeUnknown) + { + if (includeUnknown) { + final int[] vector = selector.getRowVector(); + final int[] inputSelection = mask.getSelection(); + final int inputSelectionSize = mask.getSelectionSize(); + final int[] outputSelection = match.getSelection(); + int outputSelectionSize = 0; + + for (int i = 0; i < inputSelectionSize; i++) { + final int rowNum = inputSelection[i]; + if (vector[rowNum] == nullId) { + outputSelection[outputSelectionSize++] = rowNum; + } + } + match.setSelectionSize(outputSelectionSize); + return match; + } + return VectorMatch.allFalse(); + } + }; + } + } + + /** + * Make an always false {@link VectorValueMatcher} for a {@link MultiValueDimensionVectorSelector}. When + * {@code includeUnknown} is specified to the {@link VectorValueMatcher#match(ReadableVectorMatch, boolean)} function, + * this matcher will add all rows of {@link MultiValueDimensionVectorSelector#getRowVector()} which are empty or have + * any null elements to the {@link ReadableVectorMatch} as selections, to participate in Druid 2-state logic system + * to SQL 3-state logic system conversion (as best as a multi-value dimension can). + */ + static VectorValueMatcher allFalseMultiValueDimensionMatcher(MultiValueDimensionVectorSelector selector) + { + final IdLookup idLookup = selector.idLookup(); + final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); + + if (idLookup == null || !selector.nameLookupPossibleInAdvance()) { + // must call selector.lookupName on every id to check for nulls + return new BaseVectorValueMatcher(selector) + { + @Override + public ReadableVectorMatch match(ReadableVectorMatch mask, boolean includeUnknown) + { + if (includeUnknown) { + int numRows = 0; + final IndexedInts[] vector = selector.getRowVector(); + final int[] selection = match.getSelection(); + + for (int i = 0; i < mask.getSelectionSize(); i++) { + final int rowNum = mask.getSelection()[i]; + final IndexedInts row = vector[rowNum]; + if (row.size() == 0) { + selection[numRows++] = rowNum; + } else { + final int size = row.size(); + for (int j = 0; j < size; j++) { + if (NullHandling.isNullOrEquivalent(selector.lookupName(row.get(j)))) { + selection[numRows++] = rowNum; + break; + } + } + } + } + match.setSelectionSize(numRows); + return match; + } + return VectorMatch.allFalse(); + } + }; + } else { + final int nullId = idLookup.lookupId(null); + // null value doesn't exist in column, can safely return all false matcher + if (nullId < 0) { + return ConstantMatcherType.ALL_FALSE.asVectorMatcher(selector); + } + + return new BaseVectorValueMatcher(selector) + { + @Override + public ReadableVectorMatch match(ReadableVectorMatch mask, boolean includeUnknown) + { + if (includeUnknown) { + int numRows = 0; + final IndexedInts[] vector = selector.getRowVector(); + final int[] selection = match.getSelection(); + + for (int i = 0; i < mask.getSelectionSize(); i++) { + final int rowNum = mask.getSelection()[i]; + final IndexedInts row = vector[rowNum]; + if (row.size() == 0) { + selection[numRows++] = rowNum; + } else { + final int size = row.size(); + for (int j = 0; j < size; j++) { + if (row.get(j) == nullId) { + selection[numRows++] = rowNum; + break; + } + } + } + } + match.setSelectionSize(numRows); + return match; + } + return VectorMatch.allFalse(); + } + }; + } + } + + /** + * Make an always false {@link VectorValueMatcher} for a {@link VectorValueSelector}. When {@code includeUnknown} is + * specified to the {@link VectorValueMatcher#match(ReadableVectorMatch, boolean)} function, this matcher will add + * the rows indicated as null values of {@link VectorValueSelector#getNullVector()} to the {@link ReadableVectorMatch} + * as selections, to participate in Druid 2-state logic system to SQL 3-state logic system conversion. + */ + static BaseVectorValueMatcher allFalseValueMatcher(VectorValueSelector selector) + { + return new BaseVectorValueMatcher(selector) + { + final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); + + @Override + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) + { + if (includeUnknown) { + return matchNulls(mask, match, selector.getNullVector()); + } + return VectorMatch.allFalse(); + } + }; + } + + /** + * Make an always false {@link VectorValueMatcher} for a {@link VectorObjectSelector}. When {@code includeUnknown} is + * specified to the {@link VectorValueMatcher#match(ReadableVectorMatch, boolean)} function, this matcher will add + * all rows of {@link VectorObjectSelector#getObjectVector()} which are null to the {@link ReadableVectorMatch} as + * selections, to participate in Druid 2-state logic system to SQL 3-state logic system conversion. + */ + static VectorValueMatcher allFalseObjectMatcher(VectorObjectSelector selector) + { + return new BaseVectorValueMatcher(selector) + { + final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); + + @Override + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) + { + if (includeUnknown) { + final Object[] vector = selector.getObjectVector(); + final int[] inputSelection = mask.getSelection(); + final int inputSelectionSize = mask.getSelectionSize(); + final int[] outputSelection = match.getSelection(); + int outputSelectionSize = 0; + + for (int i = 0; i < inputSelectionSize; i++) { + final int rowNum = inputSelection[i]; + if (vector[rowNum] == null) { + outputSelection[outputSelectionSize++] = rowNum; + } + } + + match.setSelectionSize(outputSelectionSize); + return match; + } + return VectorMatch.allFalse(); + } + }; + } + + static ReadableVectorMatch matchNulls( + ReadableVectorMatch mask, + VectorMatch match, + @Nullable boolean[] nullVector + ) + { + if (nullVector == null) { + return VectorMatch.allFalse(); + } + final int[] inputSelection = mask.getSelection(); + final int inputSelectionSize = mask.getSelectionSize(); + final int[] outputSelection = match.getSelection(); + int outputSelectionSize = 0; + + for (int i = 0; i < inputSelectionSize; i++) { + final int rowNum = inputSelection[i]; + if (nullVector[rowNum]) { + outputSelection[outputSelectionSize++] = rowNum; + } + } + + match.setSelectionSize(outputSelectionSize); + return match; + } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java index 01b746a839e..377aed96406 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/query/filter/vector/VectorValueMatcherFactory.java @@ -21,7 +21,6 @@ package org.apache.druid.query.filter.vector; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.vector.VectorValueSelector; import javax.annotation.Nullable; @@ -40,37 +39,4 @@ public interface VectorValueMatcherFactory VectorValueMatcher makeMatcher(Object matchValue, ColumnType matchValueType); VectorValueMatcher makeMatcher(DruidPredicateFactory predicateFactory); - - default VectorValueMatcher makeNullValueMatcher(VectorValueSelector selector) - { - return new BaseVectorValueMatcher(selector) - { - final VectorMatch match = VectorMatch.wrap(new int[selector.getMaxVectorSize()]); - - @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) - { - final boolean[] nullVector = selector.getNullVector(); - - if (nullVector == null) { - return VectorMatch.allFalse(); - } - - final int[] selection = match.getSelection(); - - int numRows = 0; - - for (int i = 0; i < mask.getSelectionSize(); i++) { - final int rowNum = mask.getSelection()[i]; - if (nullVector[rowNum]) { - selection[numRows++] = rowNum; - } - } - - match.setSelectionSize(numRows); - assert match.isValid(mask); - return match; - } - }; - } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java index 4689e37ebca..c5e561c315f 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/RowBasedGrouperHelper.java @@ -76,8 +76,8 @@ import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.data.ComparableList; import org.apache.druid.segment.data.ComparableStringArray; import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.filter.BooleanValueMatcher; import org.apache.druid.segment.filter.Filters; +import org.apache.druid.segment.filter.ValueMatchers; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -442,7 +442,7 @@ public class RowBasedGrouperHelper ); final ValueMatcher filterMatcher = filter == null - ? BooleanValueMatcher.of(true) + ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); if (subquery.getUniversalTimestamp() != null @@ -466,7 +466,7 @@ public class RowBasedGrouperHelper } } rowSupplier.set(row); - return filterMatcher.matches(); + return filterMatcher.matches(false); }; } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java b/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java index 9caf27841fc..a5fd25872bc 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/having/DimFilterHavingSpec.java @@ -109,7 +109,7 @@ public class DimFilterHavingSpec implements HavingSpec rowSupplier.set(row); } - final boolean retVal = matcher.matches(); + final boolean retVal = matcher.matches(false); if (evalCount != oldEvalCount + 1) { // Oops, someone was using this from two different threads, bad caller. diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 4a62106a557..c1a13c50441 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -318,7 +318,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns continue; } - if (!matcher.matches()) { + if (!matcher.matches(false)) { rowsToSkip.set(theId); } } diff --git a/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java b/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java index 71ba425d457..12fa165c6db 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ConcatCursor.java @@ -19,10 +19,10 @@ package org.apache.druid.query.scan; -import com.google.common.base.Predicate; import com.google.common.math.IntMath; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelectorFactory; @@ -86,12 +86,12 @@ public class ConcatCursor implements Cursor } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { return cursors.get(currentCursor) .getColumnSelectorFactory() .makeDimensionSelector(dimensionSpec) - .makeValueMatcher(predicate); + .makeValueMatcher(predicateFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java index 1f2307a6ee0..8fc431a756b 100644 --- a/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/search/UseIndexesStrategy.java @@ -181,7 +181,7 @@ public class UseIndexesStrategy extends SearchStrategy "filter[%s] should support bitmap", filter ); - baseFilter = columnIndex.computeBitmapResult(new DefaultBitmapResultFactory(selector.getBitmapFactory())); + baseFilter = columnIndex.computeBitmapResult(new DefaultBitmapResultFactory(selector.getBitmapFactory()), false); } final ImmutableBitmap timeFilteredBitmap; diff --git a/processing/src/main/java/org/apache/druid/segment/BaseSingleValueDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/BaseSingleValueDimensionSelector.java index c864e6abbc2..4e9d138f270 100644 --- a/processing/src/main/java/org/apache/druid/segment/BaseSingleValueDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/BaseSingleValueDimensionSelector.java @@ -20,6 +20,7 @@ package org.apache.druid.segment; import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -59,9 +60,10 @@ public abstract class BaseSingleValueDimensionSelector implements DimensionSelec return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return Objects.equals(getValue(), value); + final String rowValue = getValue(); + return (includeUnknown && rowValue == null) || Objects.equals(rowValue, value); } @Override @@ -73,21 +75,24 @@ public abstract class BaseSingleValueDimensionSelector implements DimensionSelec } @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { + final Predicate predicate = predicateFactory.makeStringPredicate(); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return predicate.apply(getValue()); + final String rowValue = getValue(); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + return (matchNull && rowValue == null) || predicate.apply(rowValue); } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("selector", BaseSingleValueDimensionSelector.this); - inspector.visit("predicate", predicate); + inspector.visit("predicate", predicateFactory); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/ConstantDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/ConstantDimensionSelector.java index ff719c4de7a..745aa6937f1 100644 --- a/processing/src/main/java/org/apache/druid/segment/ConstantDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/ConstantDimensionSelector.java @@ -21,11 +21,12 @@ package org.apache.druid.segment; import com.google.common.base.Predicate; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ZeroIndexedInts; -import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; import javax.annotation.Nullable; @@ -66,13 +67,14 @@ public class ConstantDimensionSelector implements SingleValueHistoricalDimension @Override public ValueMatcher makeValueMatcher(String matchValue) { - return BooleanValueMatcher.of(Objects.equals(value, matchValue)); + return Objects.equals(value, matchValue) ? ValueMatchers.allTrue() : ValueMatchers.allFalse(); } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return BooleanValueMatcher.of(predicate.apply(value)); + final Predicate predicate = predicateFactory.makeStringPredicate(); + return predicate.apply(value) ? ValueMatchers.allTrue() : ValueMatchers.allFalse(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/ConstantMultiValueDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/ConstantMultiValueDimensionSelector.java index 9025e0dc2f0..4a648717532 100644 --- a/processing/src/main/java/org/apache/druid/segment/ConstantMultiValueDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/ConstantMultiValueDimensionSelector.java @@ -20,12 +20,15 @@ package org.apache.druid.segment; import com.google.common.base.Predicate; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.RangeIndexedInts; -import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.historical.HistoricalDimensionSelector; +import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.util.List; @@ -38,6 +41,10 @@ public class ConstantMultiValueDimensionSelector implements HistoricalDimensionS public ConstantMultiValueDimensionSelector(List values) { + if (CollectionUtils.isNullOrEmpty(values)) { + throw DruidException.defensive("Use DimensionSelector.constant(null)"); + } + this.values = values; this.row = new RangeIndexedInts(); row.setSize(values.size()); @@ -98,13 +105,14 @@ public class ConstantMultiValueDimensionSelector implements HistoricalDimensionS @Override public ValueMatcher makeValueMatcher(@Nullable String value) { - return BooleanValueMatcher.of(values.stream().anyMatch(v -> Objects.equals(value, v))); + return values.stream().anyMatch(v -> Objects.equals(value, v)) ? ValueMatchers.allTrue() : ValueMatchers.allFalse(); } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return BooleanValueMatcher.of(values.stream().anyMatch(predicate::apply)); + final Predicate predicate = predicateFactory.makeStringPredicate(); + return values.stream().anyMatch(predicate::apply) ? ValueMatchers.allTrue() : ValueMatchers.allFalse(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java index 35a08969a23..5b51a29ce6a 100644 --- a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnIndexer.java @@ -20,7 +20,7 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.CloseableIndexed; @@ -174,7 +174,7 @@ public abstract class DictionaryEncodedColumnIndexer predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { throw new UnsupportedOperationException(); } diff --git a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java index bcd2ce018b5..15f7f51120d 100644 --- a/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/DictionaryEncodedColumnMerger.java @@ -19,7 +19,6 @@ package org.apache.druid.segment; -import com.google.common.base.Predicate; import com.google.common.collect.Lists; import com.google.common.collect.PeekingIterator; import it.unimi.dsi.fastutil.ints.IntIterable; @@ -32,6 +31,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; @@ -261,7 +261,7 @@ public abstract class DictionaryEncodedColumnMerger> imp } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { throw new UnsupportedOperationException(); } diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java index 8ddcd30987a..4980cf5c025 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionSelector.java @@ -23,13 +23,14 @@ import com.google.common.base.Predicate; import org.apache.druid.common.config.NullHandling; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.CalledFromHotLoop; import org.apache.druid.query.monomorphicprocessing.HotLoopCallee; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ZeroIndexedInts; -import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; import javax.annotation.Nullable; @@ -66,7 +67,7 @@ public interface DimensionSelector extends ColumnValueSelector, Dimensio */ ValueMatcher makeValueMatcher(@Nullable String value); - ValueMatcher makeValueMatcher(Predicate predicate); + ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory); /** * @deprecated This method is marked as deprecated in DimensionSelector to minimize the probability of accidental @@ -261,13 +262,21 @@ public interface DimensionSelector extends ColumnValueSelector, Dimensio @Override public ValueMatcher makeValueMatcher(@Nullable String value) { - return BooleanValueMatcher.of(value == null); + if (NullHandling.isNullOrEquivalent(value)) { + return ValueMatchers.allTrue(); + } + return ValueMatchers.allUnknown(); } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return BooleanValueMatcher.of(predicate.apply(null)); + final Predicate predicate = predicateFactory.makeStringPredicate(); + if (predicate.apply(null)) { + return ValueMatchers.allTrue(); + } + + return predicateFactory.isNullInputUnknown() ? ValueMatchers.allUnknown() : ValueMatchers.allFalse(); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/DimensionSelectorUtils.java b/processing/src/main/java/org/apache/druid/segment/DimensionSelectorUtils.java index 224b01636e8..46e77aa7a53 100644 --- a/processing/src/main/java/org/apache/druid/segment/DimensionSelectorUtils.java +++ b/processing/src/main/java/org/apache/druid/segment/DimensionSelectorUtils.java @@ -20,12 +20,12 @@ package org.apache.druid.segment; import com.google.common.base.Predicate; -import com.google.common.base.Predicates; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.filter.BooleanValueMatcher; import javax.annotation.Nullable; import java.util.BitSet; @@ -51,7 +51,7 @@ public final class DimensionSelectorUtils return makeDictionaryEncodedValueMatcherGeneric(selector, idLookup.lookupId(value), value == null); } else if (selector.getValueCardinality() >= 0 && selector.nameLookupPossibleInAdvance()) { // Employ caching BitSet optimization - return makeDictionaryEncodedValueMatcherGeneric(selector, Predicates.equalTo(value)); + return makeDictionaryEncodedValueMatcherGeneric(selector, StringPredicateDruidPredicateFactory.equalTo(value)); } else { return makeNonDictionaryEncodedValueMatcherGeneric(selector, value); } @@ -67,16 +67,17 @@ public final class DimensionSelectorUtils return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final IndexedInts row = selector.getRow(); final int size = row.size(); if (size == 0) { // null should match empty rows in multi-value columns - return matchNull; + return includeUnknown || matchNull; } else { for (int i = 0; i < size; ++i) { - if (row.get(i) == valueId) { + final int rowId = row.get(i); + if ((includeUnknown && selector.lookupName(rowId) == null) || rowId == valueId) { return true; } } @@ -91,26 +92,36 @@ public final class DimensionSelectorUtils } }; } else { - if (matchNull) { - return new ValueMatcher() + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) { - @Override - public boolean matches() - { + if (includeUnknown || matchNull) { final IndexedInts row = selector.getRow(); final int size = row.size(); - return size == 0; + if (size == 0) { + return true; + } + boolean nullRow = true; + for (int i = 0; i < size; i++) { + String rowValue = selector.lookupName(row.get(i)); + if (rowValue == null) { + return true; + } + nullRow = false; + } + return nullRow; } + return false; + } - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("selector", selector); - } - }; - } else { - return BooleanValueMatcher.of(false); - } + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; } } @@ -122,16 +133,17 @@ public final class DimensionSelectorUtils return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final IndexedInts row = selector.getRow(); final int size = row.size(); if (size == 0) { // null should match empty rows in multi-value columns - return value == null; + return includeUnknown || value == null; } else { for (int i = 0; i < size; ++i) { - if (Objects.equals(selector.lookupName(row.get(i)), value)) { + final String rowValue = selector.lookupName(row.get(i)); + if ((includeUnknown && rowValue == null) || Objects.equals(rowValue, value)) { return true; } } @@ -148,41 +160,43 @@ public final class DimensionSelectorUtils } /** - * Generic implementation of {@link DimensionSelector#makeValueMatcher(Predicate)}, uses {@link - * DimensionSelector#getRow()} of the given {@link DimensionSelector}. "Lazy" DimensionSelectors could delegate + * Generic implementation of {@link DimensionSelector#makeValueMatcher(DruidPredicateFactory)}, uses + * {@link DimensionSelector#getRow()} of the given {@link DimensionSelector}. "Lazy" DimensionSelectors could delegate * {@code makeValueMatcher()} to this method, but encouraged to implement {@code makeValueMatcher()} themselves, * bypassing the {@link IndexedInts} abstraction. */ - public static ValueMatcher makeValueMatcherGeneric(DimensionSelector selector, Predicate predicate) + public static ValueMatcher makeValueMatcherGeneric(DimensionSelector selector, DruidPredicateFactory predicateFactory) { int cardinality = selector.getValueCardinality(); if (cardinality >= 0 && selector.nameLookupPossibleInAdvance()) { - return makeDictionaryEncodedValueMatcherGeneric(selector, predicate); + return makeDictionaryEncodedValueMatcherGeneric(selector, predicateFactory); } else { - return makeNonDictionaryEncodedValueMatcherGeneric(selector, predicate); + return makeNonDictionaryEncodedValueMatcherGeneric(selector, predicateFactory); } } private static ValueMatcher makeDictionaryEncodedValueMatcherGeneric( final DimensionSelector selector, - Predicate predicate + DruidPredicateFactory predicateFactory ) { final BitSet checkedIds = new BitSet(selector.getValueCardinality()); final BitSet matchingIds = new BitSet(selector.getValueCardinality()); - final boolean matchNull = predicate.apply(null); + final Predicate predicate = predicateFactory.makeStringPredicate(); + final boolean predicateMatchesNull = predicate.apply(null); // Lazy matcher; only check an id if matches() is called. return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); final IndexedInts row = selector.getRow(); final int size = row.size(); if (size == 0) { // null should match empty rows in multi-value columns - return matchNull; + return matchNull || predicateMatchesNull; } else { for (int i = 0; i < size; ++i) { final int id = row.get(i); @@ -191,7 +205,8 @@ public final class DimensionSelectorUtils if (checkedIds.get(id)) { matches = matchingIds.get(id); } else { - matches = predicate.apply(selector.lookupName(id)); + final String rowValue = selector.lookupName(id); + matches = (matchNull && rowValue == null) || predicate.apply(rowValue); checkedIds.set(id); if (matches) { matchingIds.set(id); @@ -216,23 +231,26 @@ public final class DimensionSelectorUtils private static ValueMatcher makeNonDictionaryEncodedValueMatcherGeneric( final DimensionSelector selector, - final Predicate predicate + final DruidPredicateFactory predicateFactory ) { - final boolean matchNull = predicate.apply(null); + final Predicate predicate = predicateFactory.makeStringPredicate(); + final boolean predicateMatchesNull = predicate.apply(null); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); final IndexedInts row = selector.getRow(); final int size = row.size(); if (size == 0) { // null should match empty rows in multi-value columns - return matchNull; + return matchNull || predicateMatchesNull; } else { for (int i = 0; i < size; ++i) { - if (predicate.apply(selector.lookupName(row.get(i)))) { + final String rowValue = selector.lookupName(row.get(i)); + if ((matchNull && rowValue == null) || predicate.apply(rowValue)) { return true; } } diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java index f6103c3ce2f..e7c9f96c65b 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java @@ -29,7 +29,7 @@ import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.Offset; import org.apache.druid.segment.data.ReadableOffset; -import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.index.BitmapColumnIndex; import org.roaringbitmap.IntIterator; @@ -63,7 +63,10 @@ public final class FilteredOffset extends Offset // offset and must use the value matcher here if (columnIndex != null && columnIndex.getIndexCapabilities().isExact()) { filterMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher( - columnIndex.computeBitmapResult(new DefaultBitmapResultFactory(bitmapIndexSelector.getBitmapFactory())) + columnIndex.computeBitmapResult( + new DefaultBitmapResultFactory(bitmapIndexSelector.getBitmapFactory()), + false + ) ); } else { filterMatcher = postFilter.makeMatcher(columnSelectorFactory); @@ -77,7 +80,7 @@ public final class FilteredOffset extends Offset { while (!Thread.currentThread().isInterrupted()) { baseOffset.increment(); - if (!baseOffset.withinBounds() || filterMatcher.matches()) { + if (!baseOffset.withinBounds() || filterMatcher.matches(false)) { return; } } @@ -99,7 +102,7 @@ public final class FilteredOffset extends Offset private void incrementIfNeededOnCreationOrReset() { if (baseOffset.withinBounds()) { - if (!filterMatcher.matches()) { + if (!filterMatcher.matches(false)) { increment(); // increment() returns early if it detects the current Thread is interrupted. It will leave this // FilteredOffset in an illegal state, because it may point to an offset that should be filtered. So must to @@ -165,7 +168,7 @@ public final class FilteredOffset extends Offset rowBitmap.iterator(); if (!iter.hasNext()) { - return BooleanValueMatcher.of(false); + return ValueMatchers.allFalse(); } if (descending) { @@ -174,7 +177,7 @@ public final class FilteredOffset extends Offset int iterOffset = Integer.MAX_VALUE; @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { int currentOffset = offset.getOffset(); while (iterOffset > currentOffset && iter.hasNext()) { @@ -197,7 +200,7 @@ public final class FilteredOffset extends Offset int iterOffset = -1; @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { int currentOffset = offset.getOffset(); while (iterOffset < currentOffset && iter.hasNext()) { diff --git a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java index 05c721e5c56..6b1ff6b3195 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexMergerV9.java @@ -202,6 +202,7 @@ public class IndexMergerV9 implements IndexMerger mergers.add( handler.makeMerger( indexSpec, + segmentWriteOutMedium, dimFormats.get(i).toColumnCapabilities(), progress, diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java index 2a4fe4d5c72..d0712e9c315 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedColumnSelectorFactory.java @@ -25,6 +25,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.Rows; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; @@ -232,16 +233,17 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { updateCurrentValues(); if (dimensionValues.isEmpty()) { - return value == null; + return includeUnknown || value == null; } for (String dimensionValue : dimensionValues) { - if (Objects.equals(NullHandling.emptyToNullIfNeeded(dimensionValue), value)) { + final String coerced = NullHandling.emptyToNullIfNeeded(dimensionValue); + if ((includeUnknown && coerced == null) || Objects.equals(coerced, value)) { return true; } } @@ -258,23 +260,26 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory } @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { - final boolean matchNull = predicate.apply(null); + final Predicate predicate = predicateFactory.makeStringPredicate(); + final boolean predicateMatchNull = predicate.apply(null); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); updateCurrentValues(); if (dimensionValues.isEmpty()) { - return matchNull; + return matchNull || predicateMatchNull; } for (String dimensionValue : dimensionValues) { - if (predicate.apply(NullHandling.emptyToNullIfNeeded(dimensionValue))) { + final String coerced = NullHandling.emptyToNullIfNeeded(dimensionValue); + if ((matchNull && coerced == null) || predicate.apply(coerced)) { return true; } } @@ -285,7 +290,7 @@ public class RowBasedColumnSelectorFactory implements ColumnSelectorFactory public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("row", rowSupplier); - inspector.visit("predicate", predicate); + inspector.visit("predicate", predicateFactory); inspector.visit("extractionFn", extractionFn); } }; diff --git a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java index eb7a870c172..f843fe63131 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/RowBasedCursor.java @@ -24,7 +24,7 @@ import org.apache.druid.query.BaseQuery; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.filter.ValueMatchers; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -76,7 +76,7 @@ public class RowBasedCursor implements Cursor ); if (filter == null) { - this.valueMatcher = BooleanValueMatcher.of(true); + this.valueMatcher = ValueMatchers.allTrue(); } else { this.valueMatcher = filter.makeMatcher(this.columnSelectorFactory); } @@ -135,7 +135,7 @@ public class RowBasedCursor implements Cursor private void advanceToMatchingRow() { - while (!isDone() && !valueMatcher.matches()) { + while (!isDone() && !valueMatcher.matches(false)) { rowWalker.advance(); rowId++; } diff --git a/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java index 1cf3efb2458..615f356f544 100644 --- a/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/SingleScanTimeDimensionSelector.java @@ -22,6 +22,7 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.IndexedInts; @@ -77,9 +78,10 @@ public class SingleScanTimeDimensionSelector implements DimensionSelector return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return Objects.equals(lookupName(getDimensionValueIndex()), value); + final String rowVal = lookupName(getDimensionValueIndex()); + return (includeUnknown && rowVal == null) || Objects.equals(rowVal, value); } @Override @@ -91,21 +93,24 @@ public class SingleScanTimeDimensionSelector implements DimensionSelector } @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { + final Predicate predicate = predicateFactory.makeStringPredicate(); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return predicate.apply(lookupName(getDimensionValueIndex())); + final String rowVal = lookupName(getDimensionValueIndex()); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + return (matchNull && rowVal == null) || predicate.apply(lookupName(getDimensionValueIndex())); } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("selector", SingleScanTimeDimensionSelector.this); - inspector.visit("predicate", predicate); + inspector.visit("predicate", predicateFactory); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java index f1a10dc3f0b..b887077027e 100644 --- a/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java +++ b/processing/src/main/java/org/apache/druid/segment/StringDimensionIndexer.java @@ -20,7 +20,6 @@ package org.apache.druid.segment; import com.google.common.base.Predicate; -import com.google.common.base.Predicates; import com.google.common.primitives.Ints; import it.unimi.dsi.fastutil.ints.IntArrays; import org.apache.druid.collections.bitmap.BitmapFactory; @@ -32,6 +31,8 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; @@ -39,7 +40,6 @@ import org.apache.druid.segment.column.ColumnCapabilitiesImpl; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.ArrayBasedIndexedInts; import org.apache.druid.segment.data.IndexedInts; -import org.apache.druid.segment.filter.BooleanValueMatcher; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexRow; import org.apache.druid.segment.incremental.IncrementalIndexRowHolder; @@ -349,26 +349,30 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer= 0 || value == null) { return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length) { - return value == null; + return includeUnknown || value == null; } int[] dimsInt = (int[]) dims[dimIndex]; if (dimsInt == null || dimsInt.length == 0) { - return value == null; + return includeUnknown || value == null; } for (int id : dimsInt) { if (id == valueId) { return true; } + if (includeUnknown && (id == nullValueId)) { + return true; + } } return false; } @@ -380,38 +384,70 @@ public class StringDimensionIndexer extends DictionaryEncodedColumnIndexer predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { final BitSet checkedIds = new BitSet(maxId); final BitSet matchingIds = new BitSet(maxId); - final boolean matchNull = predicate.apply(null); + final Predicate predicate = predicateFactory.makeStringPredicate(); + final boolean predicateMatchesNull = predicate.apply(null); + final int nullValueId = lookupId(null); // Lazy matcher; only check an id if matches() is called. return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); Object[] dims = currEntry.get().getDims(); if (dimIndex >= dims.length) { - return matchNull; + return matchNull || predicateMatchesNull; } int[] dimsInt = (int[]) dims[dimIndex]; if (dimsInt == null || dimsInt.length == 0) { - return matchNull; + return matchNull || predicateMatchesNull; } for (int id : dimsInt) { + if (includeUnknown && id == nullValueId) { + checkedIds.set(id); + return true; + } if (checkedIds.get(id)) { if (matchingIds.get(id)) { return true; diff --git a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java index d69eca6109d..5db2dd414e0 100644 --- a/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/UnnestDimensionCursor.java @@ -19,10 +19,10 @@ package org.apache.druid.segment; -import com.google.common.base.Predicate; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; @@ -124,9 +124,9 @@ public class UnnestDimensionCursor implements Cursor return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return false; + return includeUnknown; } @Override @@ -140,15 +140,16 @@ public class UnnestDimensionCursor implements Cursor return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { if (indexedIntsForCurrentRow == null) { - return false; + return includeUnknown; } if (indexedIntsForCurrentRow.size() <= 0) { - return false; + return includeUnknown; } - return idForLookup == indexedIntsForCurrentRow.get(index); + final int rowId = indexedIntsForCurrentRow.get(index); + return (includeUnknown && lookupName(rowId) == null) || idForLookup == rowId; } @Override @@ -160,9 +161,9 @@ public class UnnestDimensionCursor implements Cursor } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java index 63f2b36313c..8fd1673dfcc 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java @@ -20,9 +20,11 @@ package org.apache.druid.segment.column; import com.google.common.base.Predicate; -import com.google.common.base.Predicates; +import com.google.common.collect.Lists; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; @@ -34,7 +36,7 @@ import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.data.SingleIndexedInt; -import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.filter.ValueMatchers; import org.apache.druid.segment.historical.HistoricalDimensionSelector; import org.apache.druid.segment.historical.SingleValueHistoricalDimensionSelector; import org.apache.druid.segment.nested.NestedCommonFormatColumn; @@ -48,7 +50,6 @@ import org.apache.druid.utils.CloseableUtils; import javax.annotation.Nullable; import java.io.IOException; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.BitSet; import java.util.List; @@ -216,9 +217,9 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Nullable @@ -279,13 +280,15 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum { if (extractionFn == null) { final int valueId = super.lookupId(value); + final int nullId = super.lookupId(null); if (valueId >= 0) { return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return getRowValue() == valueId; + final int rowId = getRowValue(); + return (includeUnknown && rowId == nullId) || rowId == valueId; } @Override @@ -295,32 +298,55 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum } }; } else { - return BooleanValueMatcher.of(false); + // no nulls, and value isn't in column, we can safely optimize to 'allFalse' + if (nullId < 0) { + return ValueMatchers.allFalse(); + } + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + if (includeUnknown && getRowValue() == 0) { + return true; + } + return false; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", StringUtf8DictionaryEncodedColumn.this); + } + }; } } else { // Employ caching BitSet optimization - return makeValueMatcher(Predicates.equalTo(value)); + return makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(value)); } } @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { final BitSet checkedIds = new BitSet(getCardinality()); final BitSet matchingIds = new BitSet(getCardinality()); + final Predicate predicate = predicateFactory.makeStringPredicate(); // Lazy matcher; only check an id if matches() is called. return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final int id = getRowValue(); if (checkedIds.get(id)) { return matchingIds.get(id); } else { - final boolean matches = predicate.apply(lookupName(id)); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + final String rowValue = lookupName(id); + final boolean matches = (matchNull && rowValue == null) || predicate.apply(rowValue); checkedIds.set(id); if (matches) { matchingIds.set(id); @@ -772,14 +798,14 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum for (int i = 0; i < offset.getCurrentVectorSize(); i++) { IndexedInts ithRow = vector[i]; - if (ithRow.size() == 0) { + final int size = ithRow.size(); + if (size == 0) { strings[i] = null; - } else if (ithRow.size() == 1) { + } else if (size == 1) { strings[i] = lookupName(ithRow.get(0)); } else { - List row = new ArrayList<>(ithRow.size()); - // noinspection SSBasedInspection - for (int j = 0; j < ithRow.size(); j++) { + List row = Lists.newArrayListWithCapacity(size); + for (int j = 0; j < size; j++) { row.add(lookupName(ithRow.get(j))); } strings[i] = row; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/FalseValueMatcher.java b/processing/src/main/java/org/apache/druid/segment/filter/AllFalseValueMatcher.java similarity index 76% rename from processing/src/main/java/org/apache/druid/segment/filter/FalseValueMatcher.java rename to processing/src/main/java/org/apache/druid/segment/filter/AllFalseValueMatcher.java index 97d12562b2e..f373c81e67f 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/FalseValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AllFalseValueMatcher.java @@ -22,21 +22,20 @@ package org.apache.druid.segment.filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -final class FalseValueMatcher implements ValueMatcher +/** + * Constant condition {@link ValueMatcher} that always returns false, for columns with no null 'unknown' values + */ +final class AllFalseValueMatcher implements ValueMatcher { - private static final FalseValueMatcher INSTANCE = new FalseValueMatcher(); + private static final AllFalseValueMatcher INSTANCE = new AllFalseValueMatcher(); - public static FalseValueMatcher instance() + public static AllFalseValueMatcher instance() { return INSTANCE; } - private FalseValueMatcher() - { - } - @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { return false; } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/TrueValueMatcher.java b/processing/src/main/java/org/apache/druid/segment/filter/AllTrueValueMatcher.java similarity index 75% rename from processing/src/main/java/org/apache/druid/segment/filter/TrueValueMatcher.java rename to processing/src/main/java/org/apache/druid/segment/filter/AllTrueValueMatcher.java index 953c255af56..58829c5c487 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/TrueValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AllTrueValueMatcher.java @@ -22,21 +22,24 @@ package org.apache.druid.segment.filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; -final class TrueValueMatcher implements ValueMatcher +/** + * Constant condition {@link ValueMatcher} that always returns true regardless of the underlying column value + */ +final class AllTrueValueMatcher implements ValueMatcher { - private static final TrueValueMatcher INSTANCE = new TrueValueMatcher(); + private static final AllTrueValueMatcher INSTANCE = new AllTrueValueMatcher(); - public static TrueValueMatcher instance() + public static AllTrueValueMatcher instance() { return INSTANCE; } - private TrueValueMatcher() + private AllTrueValueMatcher() { } @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { return true; } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/BooleanValueMatcher.java b/processing/src/main/java/org/apache/druid/segment/filter/AllUnknownValueMatcher.java similarity index 56% rename from processing/src/main/java/org/apache/druid/segment/filter/BooleanValueMatcher.java rename to processing/src/main/java/org/apache/druid/segment/filter/AllUnknownValueMatcher.java index 1a65aef74c8..404d10f8c81 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/BooleanValueMatcher.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AllUnknownValueMatcher.java @@ -20,17 +20,34 @@ package org.apache.druid.segment.filter; import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; /** -*/ -public final class BooleanValueMatcher + * Constant condition {@link ValueMatcher} that always returns the boolean value passed to {@link #matches(boolean)}, + * for columns which contain only null values + */ +final class AllUnknownValueMatcher implements ValueMatcher { - public static ValueMatcher of(boolean matches) + private static final AllUnknownValueMatcher INSTANCE = new AllUnknownValueMatcher(); + + public static AllUnknownValueMatcher instance() { - return matches ? TrueValueMatcher.instance() : FalseValueMatcher.instance(); + return INSTANCE; } - private BooleanValueMatcher() + private AllUnknownValueMatcher() { } + + @Override + public boolean matches(boolean includeUnknown) + { + return includeUnknown; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // nothing to inspect + } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java index d411589d99e..20dc33b8842 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java @@ -53,7 +53,7 @@ import java.util.Map; import java.util.Objects; /** - * + * Logical AND filter operation */ public class AndFilter implements BooleanFilter { @@ -80,7 +80,7 @@ public class AndFilter implements BooleanFilter ) { return bitmapResultFactory.toImmutableBitmap( - getBitmapIndex(selector, filters).computeBitmapResult(bitmapResultFactory) + getBitmapIndex(selector, filters).computeBitmapResult(bitmapResultFactory, false) ); } @@ -126,11 +126,11 @@ public class AndFilter implements BooleanFilter } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { final List bitmapResults = new ArrayList<>(bitmapColumnIndices.size()); for (final BitmapColumnIndex index : bitmapColumnIndices) { - final T bitmapResult = index.computeBitmapResult(bitmapResultFactory); + final T bitmapResult = index.computeBitmapResult(bitmapResultFactory, includeUnknown); if (bitmapResultFactory.isEmpty(bitmapResult)) { // Short-circuit. return bitmapResultFactory.wrapAllFalse( @@ -196,7 +196,7 @@ public class AndFilter implements BooleanFilter for (Filter filter : filters) { final BitmapColumnIndex columnIndex = filter.getBitmapColumnIndex(selector); if (columnIndex != null && columnIndex.getIndexCapabilities().isExact()) { - bitmaps.add(columnIndex.computeBitmapResult(resultFactory)); + bitmaps.add(columnIndex.computeBitmapResult(resultFactory, false)); } else { ValueMatcher matcher = filter.makeMatcher(columnSelectorFactory); matchers.add(matcher); @@ -256,10 +256,10 @@ public class AndFilter implements BooleanFilter return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { for (ValueMatcher matcher : baseMatchers) { - if (!matcher.matches()) { + if (!matcher.matches(includeUnknown)) { return false; } } @@ -287,7 +287,7 @@ public class AndFilter implements BooleanFilter return new BaseVectorValueMatcher(baseMatchers[0]) { @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { ReadableVectorMatch match = mask; @@ -297,7 +297,7 @@ public class AndFilter implements BooleanFilter break; } - match = matcher.match(match); + match = matcher.match(match, includeUnknown); } assert match.isValid(mask); diff --git a/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java index d08160a3166..a75da57654e 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/BoundFilter.java @@ -46,8 +46,8 @@ import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; @@ -85,7 +85,7 @@ public class BoundFilter implements Filter // missing column -> match all rows if the predicate matches null; match no rows otherwise return getPredicateFactory().makeStringPredicate().apply(null) ? new AllTrueBitmapColumnIndex(selector) - : new AllFalseBitmapColumnIndex(selector); + : new AllUnknownBitmapColumnIndex(selector); } if (supportStringShortCircuit()) { @@ -179,12 +179,12 @@ public class BoundFilter implements Filter } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { return bitmapResultFactory.union( ImmutableList.of( - rangeIndex.computeBitmapResult(bitmapResultFactory), - nullBitmap.computeBitmapResult(bitmapResultFactory) + rangeIndex.computeBitmapResult(bitmapResultFactory, false), + nullBitmap.computeBitmapResult(bitmapResultFactory, false) ) ); } @@ -343,6 +343,7 @@ public class BoundFilter implements Filter private final Supplier longPredicateSupplier; private final Supplier floatPredicateSupplier; private final Supplier doublePredicateSupplier; + private final boolean isNullUnknown; BoundDimFilterDruidPredicateFactory(ExtractionFn extractionFn, BoundDimFilter boundDimFilter) { @@ -351,6 +352,11 @@ public class BoundFilter implements Filter this.longPredicateSupplier = boundDimFilter.getLongPredicateSupplier(); this.floatPredicateSupplier = boundDimFilter.getFloatPredicateSupplier(); this.doublePredicateSupplier = boundDimFilter.getDoublePredicateSupplier(); + if (extractionFn != null) { + this.isNullUnknown = !doesMatch(extractionFn.apply(null), boundDimFilter); + } else { + this.isNullUnknown = !doesMatch(null, boundDimFilter); + } } @Override @@ -399,6 +405,12 @@ public class BoundFilter implements Filter return input -> doesMatch(String.valueOf(input), boundDimFilter); } + @Override + public boolean isNullInputUnknown() + { + return isNullUnknown; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java index 9be84678652..ed1590dc16e 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ColumnComparisonFilter.java @@ -48,6 +48,11 @@ import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; +/** + * Compares values between columns, first converting them all to strings. This filter behaves like "not distinct from", + * e.g. given columns x and y, the SQL equivalent would be "x is not distinct from y" (and so ignores + * {@code includeUnknown}). + */ public class ColumnComparisonFilter implements Filter { private final List dimensions; @@ -79,13 +84,13 @@ public class ColumnComparisonFilter implements Filter public static ValueMatcher makeValueMatcher(final List> valueGetters) { if (valueGetters.isEmpty()) { - return BooleanValueMatcher.of(true); + return ValueMatchers.allTrue(); } return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { // Keep all values to compare against each other. String[][] values = new String[valueGetters.size()][]; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ConstantMatcherType.java b/processing/src/main/java/org/apache/druid/segment/filter/ConstantMatcherType.java new file mode 100644 index 00000000000..ee3decb12a5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/filter/ConstantMatcherType.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.filter; + +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.filter.vector.BooleanVectorValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.segment.vector.VectorSizeInspector; + +public enum ConstantMatcherType +{ + /** + * Constant matcher that is always true + */ + ALL_TRUE { + @Override + public ValueMatcher asValueMatcher() + { + return ValueMatchers.allTrue(); + } + + @Override + public VectorValueMatcher asVectorMatcher(VectorSizeInspector inspector) + { + return BooleanVectorValueMatcher.of(inspector, this); + } + }, + /** + * Constant matcher that is always false + */ + ALL_FALSE { + @Override + public ValueMatcher asValueMatcher() + { + return ValueMatchers.allFalse(); + } + + @Override + public VectorValueMatcher asVectorMatcher(VectorSizeInspector inspector) + { + return BooleanVectorValueMatcher.of(inspector, this); + } + }, + /** + * Constant matcher that is always null + */ + ALL_UNKNOWN { + @Override + public ValueMatcher asValueMatcher() + { + return ValueMatchers.allUnknown(); + } + + @Override + public VectorValueMatcher asVectorMatcher(VectorSizeInspector inspector) + { + return BooleanVectorValueMatcher.of(inspector, this); + } + }; + + public abstract ValueMatcher asValueMatcher(); + + public abstract VectorValueMatcher asVectorMatcher(VectorSizeInspector inspector); +} diff --git a/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java index e5e6e9563a7..f321691e033 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/DimensionPredicateFilter.java @@ -146,12 +146,14 @@ public class DimensionPredicateFilter implements Filter private final Predicate baseStringPredicate; private final DruidPredicateFactory predicateFactory; private final ExtractionFn extractionFn; + private final boolean isNullUnknown; DelegatingStringPredicateFactory(DruidPredicateFactory predicateFactory, ExtractionFn extractionFn) { this.predicateFactory = predicateFactory; this.baseStringPredicate = predicateFactory.makeStringPredicate(); this.extractionFn = extractionFn; + this.isNullUnknown = !baseStringPredicate.apply(extractionFn.apply(null)); } @Override @@ -217,6 +219,12 @@ public class DimensionPredicateFilter implements Filter }; } + @Override + public boolean isNullInputUnknown() + { + return isNullUnknown; + } + @Override public boolean equals(Object o) { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java index 711395cfe82..6d2bfab4b6a 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java @@ -38,7 +38,6 @@ import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.filter.vector.BooleanVectorValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -92,19 +91,15 @@ public class ExpressionFilter implements Filter // input type information, so composed entirely of null constants or missing columns. the expression is // effectively constant if (outputType == null) { - - // in sql compatible mode, this means no matches ever because null doesn't equal anything so just use the - // false matcher - if (NullHandling.sqlCompatible()) { - return BooleanVectorValueMatcher.of(factory.getReadableVectorInspector(), false); + // evaluate the expression, just in case it does actually match nulls + final ExprEval constantEval = theExpr.eval(InputBindings.nilBindings()); + final ConstantMatcherType constantMatcherType; + if (constantEval.value() == null) { + constantMatcherType = ConstantMatcherType.ALL_UNKNOWN; + } else { + constantMatcherType = constantEval.asBoolean() ? ConstantMatcherType.ALL_TRUE : ConstantMatcherType.ALL_FALSE; } - // however in default mode, we still need to evaluate the expression since it might end up... strange, from - // default values. Since it is effectively constant though, we can just do that up front and decide if it matches - // or not. - return BooleanVectorValueMatcher.of( - factory.getReadableVectorInspector(), - theExpr.eval(InputBindings.nilBindings()).asBoolean() - ); + return constantMatcherType.asVectorMatcher(factory.getReadableVectorInspector()); } // if we got here, we really have to evaluate the expressions to match @@ -147,10 +142,14 @@ public class ExpressionFilter implements Filter return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final ExprEval eval = selector.getObject(); + if (includeUnknown && eval.value() == null) { + return true; + } + if (eval.type().isArray()) { switch (eval.elementType().getType()) { case LONG: @@ -158,6 +157,9 @@ public class ExpressionFilter implements Filter if (lResult == null) { return false; } + if (includeUnknown) { + return Arrays.stream(lResult).anyMatch(o -> o == null || Evals.asBoolean((long) o)); + } return Arrays.stream(lResult).filter(Objects::nonNull).anyMatch(o -> Evals.asBoolean((long) o)); case STRING: @@ -166,6 +168,10 @@ public class ExpressionFilter implements Filter return false; } + if (includeUnknown) { + return Arrays.stream(sResult).anyMatch(o -> o == null || Evals.asBoolean((String) o)); + } + return Arrays.stream(sResult).anyMatch(o -> Evals.asBoolean((String) o)); case DOUBLE: final Object[] dResult = eval.asArray(); @@ -173,10 +179,14 @@ public class ExpressionFilter implements Filter return false; } + if (includeUnknown) { + return Arrays.stream(dResult).anyMatch(o -> o == null || Evals.asBoolean((double) o)); + } + return Arrays.stream(dResult).filter(Objects::nonNull).anyMatch(o -> Evals.asBoolean((double) o)); } } - return eval.asBoolean(); + return (includeUnknown && eval.value() == null) || eval.asBoolean(); } @Override @@ -287,6 +297,7 @@ public class ExpressionFilter implements Filter */ private DruidPredicateFactory getPredicateFactory() { + final boolean isNullUnknown = expr.get().eval(InputBindings.nilBindings()).value() == null; return new DruidPredicateFactory() { @Override @@ -335,6 +346,12 @@ public class ExpressionFilter implements Filter { return super.equals(obj); } + + @Override + public boolean isNullInputUnknown() + { + return isNullUnknown; + } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java index cbc2b21b287..ba63984fd2e 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/FalseFilter.java @@ -22,7 +22,6 @@ package org.apache.druid.segment.filter; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.filter.vector.BooleanVectorValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelector; @@ -53,7 +52,7 @@ public class FalseFilter implements Filter @Override public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) { - return new AllFalseBitmapColumnIndex(selector); + return new AllFalseBitmapColumnIndex(selector.getBitmapFactory()); } @Override @@ -65,13 +64,13 @@ public class FalseFilter implements Filter @Override public ValueMatcher makeMatcher(ColumnSelectorFactory factory) { - return FalseValueMatcher.instance(); + return ConstantMatcherType.ALL_FALSE.asValueMatcher(); } @Override public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) { - return BooleanVectorValueMatcher.of(factory.getReadableVectorInspector(), false); + return ConstantMatcherType.ALL_FALSE.asVectorMatcher(factory.getReadableVectorInspector()); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/filter/Filters.java b/processing/src/main/java/org/apache/druid/segment/filter/Filters.java index 14c2e2f770c..54a20461ce9 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/Filters.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/Filters.java @@ -41,8 +41,8 @@ import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; import org.apache.druid.segment.filter.cnf.CalciteCnfHelper; import org.apache.druid.segment.filter.cnf.HiveCnfHelper; -import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; @@ -142,18 +142,19 @@ public class Filters // missing column -> match all rows if the predicate matches null; match no rows otherwise return predicateFactory.makeStringPredicate().apply(null) ? new AllTrueBitmapColumnIndex(selector) - : new AllFalseBitmapColumnIndex(selector); + : new AllUnknownBitmapColumnIndex(selector); } public static BitmapColumnIndex makeMissingColumnNullIndex(boolean matchesNull, final ColumnIndexSelector selector) { - return matchesNull ? new AllTrueBitmapColumnIndex(selector) : new AllFalseBitmapColumnIndex(selector); + return matchesNull ? new AllTrueBitmapColumnIndex(selector) : new AllUnknownBitmapColumnIndex(selector); } public static ImmutableBitmap computeDefaultBitmapResults(Filter filter, ColumnIndexSelector selector) { return filter.getBitmapColumnIndex(selector).computeBitmapResult( - new DefaultBitmapResultFactory(selector.getBitmapFactory()) + new DefaultBitmapResultFactory(selector.getBitmapFactory()), + false ); } @@ -371,7 +372,7 @@ public class Filters public static boolean filterMatchesNull(Filter filter) { ValueMatcher valueMatcher = filter.makeMatcher(ALL_NULL_COLUMN_SELECTOR_FACTORY); - return valueMatcher.matches(); + return valueMatcher.matches(false); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java index ff3f9b51e96..6a474969326 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java @@ -35,8 +35,8 @@ import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnIndexSupplier; -import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; import org.apache.druid.segment.index.AllTrueBitmapColumnIndex; +import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; import org.apache.druid.segment.index.semantic.StringValueSetIndexes; @@ -79,7 +79,7 @@ public class LikeFilter implements Filter // Treat this as a column full of nulls return likeMatcher.matches(null) ? new AllTrueBitmapColumnIndex(selector) - : new AllFalseBitmapColumnIndex(selector); + : new AllUnknownBitmapColumnIndex(selector); } if (isSimpleEquals()) { StringValueSetIndexes valueIndexes = indexSupplier.as(StringValueSetIndexes.class); diff --git a/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java index 2201c06c441..389c9ceae36 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java @@ -19,7 +19,9 @@ package org.apache.druid.segment.filter; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.Filter; @@ -42,6 +44,18 @@ import java.util.Objects; import java.util.Set; /** + * Nice filter you have there... NOT! + * + * If {@link ExpressionProcessing#useStrictBooleans()} and {@link NullHandling#sqlCompatible()} are both true, this + * filter inverts the {@code includeUnknown} flag to properly map Druids native two-valued logic (true, false) to SQL + * three-valued logic (true, false, unknown). At the top level, this flag is always passed in as 'false', and is only + * flipped by this filter. Other logical filters ({@link AndFilter} and {@link OrFilter}) propagate the value of + * {@code includeUnknown} to their children. + * + * For example, if the base filter is equality, by default value matchers and indexes only return true for the rows + * that are equal to the value. When wrapped in a not filter, the not filter indicates that the equality matchers and + * indexes should also include the null or 'unknown' values as matches, so that inverting the match does not incorrectly + * include these null values as matches. */ public class NotFilter implements Filter { @@ -60,6 +74,7 @@ public class NotFilter implements Filter if (baseIndex != null && baseIndex.getIndexCapabilities().isInvertible()) { return new BitmapColumnIndex() { + private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic(); @Override public ColumnIndexCapabilities getIndexCapabilities() { @@ -73,10 +88,10 @@ public class NotFilter implements Filter } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { return bitmapResultFactory.complement( - baseIndex.computeBitmapResult(bitmapResultFactory), + baseIndex.computeBitmapResult(bitmapResultFactory, !includeUnknown && useThreeValueLogic), selector.getNumRows() ); } @@ -92,10 +107,11 @@ public class NotFilter implements Filter return new ValueMatcher() { + private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic(); @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return !baseMatcher.matches(); + return !baseMatcher.matches(!includeUnknown && useThreeValueLogic); } @Override @@ -113,12 +129,13 @@ public class NotFilter implements Filter return new BaseVectorValueMatcher(baseMatcher) { - final VectorMatch scratch = VectorMatch.wrap(new int[factory.getMaxVectorSize()]); + private final VectorMatch scratch = VectorMatch.wrap(new int[factory.getMaxVectorSize()]); + private final boolean useThreeValueLogic = NullHandling.useThreeValueLogic(); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { - final ReadableVectorMatch baseMatch = baseMatcher.match(mask); + final ReadableVectorMatch baseMatch = baseMatcher.match(mask, !includeUnknown && useThreeValueLogic); scratch.copyFrom(mask); scratch.removeAll(baseMatch); diff --git a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java index f5840d3d0c8..2426a9882ce 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java @@ -52,7 +52,7 @@ import java.util.Map; import java.util.Objects; /** - * + * Logical OR filter operation */ public class OrFilter implements BooleanFilter { @@ -112,10 +112,10 @@ public class OrFilter implements BooleanFilter } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { return bitmapResultFactory.union( - () -> bitmapColumnIndices.stream().map(x -> x.computeBitmapResult(bitmapResultFactory)).iterator() + () -> bitmapColumnIndices.stream().map(x -> x.computeBitmapResult(bitmapResultFactory, includeUnknown)).iterator() ); } }; @@ -166,7 +166,7 @@ public class OrFilter implements BooleanFilter for (Filter filter : filters) { final BitmapColumnIndex columnIndex = filter.getBitmapColumnIndex(selector); if (columnIndex != null && columnIndex.getIndexCapabilities().isExact()) { - bitmaps.add(columnIndex.computeBitmapResult(resultFactory)); + bitmaps.add(columnIndex.computeBitmapResult(resultFactory, false)); } else { ValueMatcher matcher = filter.makeMatcher(columnSelectorFactory); matchers.add(matcher); @@ -227,10 +227,10 @@ public class OrFilter implements BooleanFilter return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { for (ValueMatcher matcher : baseMatchers) { - if (matcher.matches()) { + if (matcher.matches(includeUnknown)) { return true; } } @@ -262,9 +262,9 @@ public class OrFilter implements BooleanFilter final VectorMatch retVal = VectorMatch.wrap(new int[getMaxVectorSize()]); @Override - public ReadableVectorMatch match(final ReadableVectorMatch mask) + public ReadableVectorMatch match(final ReadableVectorMatch mask, boolean includeUnknown) { - ReadableVectorMatch currentMatch = baseMatchers[0].match(mask); + ReadableVectorMatch currentMatch = baseMatchers[0].match(mask, includeUnknown); // Initialize currentMask = mask, then progressively remove rows from the mask as we find matches for them. // This isn't necessary for correctness (we could use the original "mask" on every call to "match") but it @@ -282,7 +282,7 @@ public class OrFilter implements BooleanFilter } currentMask.removeAll(currentMatch); - currentMatch = baseMatchers[i].match(currentMask); + currentMatch = baseMatchers[i].match(currentMask, false); retVal.addAll(currentMatch, scratch); if (currentMatch == currentMask) { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java index 36b97d6d4d5..796177a4c79 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/PredicateValueMatcherFactory.java @@ -94,18 +94,24 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory predicate = predicateFactory.makeArrayPredicate(columnCapabilities); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); Object o = selector.getObject(); if (o == null || o instanceof Object[]) { - return predicate.apply((Object[]) o); + return (matchNull && o == null) || predicate.apply((Object[]) o); } if (o instanceof List) { ExprEval oEval = ExprEval.bestEffortArray((List) o); @@ -130,16 +136,22 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory predicate = predicateFactory.makeObjectPredicate(); return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return predicate.apply(selector.getObject()); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + final Object val = selector.getObject(); + return (matchNull && val == null) || predicate.apply(val); } @Override @@ -162,12 +174,13 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory arrayPredicate; @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final Object rowValue = selector.getObject(); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); if (rowValue == null) { - return getStringPredicate().apply(null); + return matchNull || getStringPredicate().apply(null); } else if (rowValue instanceof Integer) { return getLongPredicate().applyLong((int) rowValue); } else if (rowValue instanceof Long) { @@ -186,11 +199,12 @@ public class PredicateValueMatcherFactory implements ColumnProcessorFactory T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { Iterable search = spatialIndex.getRTree().search(bound); return bitmapResultFactory.unionDimensionValueBitmaps(search); diff --git a/processing/src/main/java/org/apache/druid/segment/filter/StringConstantValueMatcherFactory.java b/processing/src/main/java/org/apache/druid/segment/filter/StringConstantValueMatcherFactory.java index a1a722e2527..d4fd0a87e21 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/StringConstantValueMatcherFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/StringConstantValueMatcherFactory.java @@ -84,6 +84,7 @@ public class StringConstantValueMatcherFactory implements ColumnProcessorFactory ) { // this is gonna fail because SelectorPredicateFactory does not implement array predicate... + // let it happen anyway to bubble up standard error messaging about predicate not supporting arrays return new PredicateValueMatcherFactory( new SelectorPredicateFactory(matchValue) ).makeArrayProcessor(selector, columnCapabilities); diff --git a/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java index b363b384331..565e0db6b9d 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/TrueFilter.java @@ -22,7 +22,6 @@ package org.apache.druid.segment.filter; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; -import org.apache.druid.query.filter.vector.BooleanVectorValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelector; @@ -62,13 +61,13 @@ public class TrueFilter implements Filter @Override public ValueMatcher makeMatcher(ColumnSelectorFactory factory) { - return BooleanValueMatcher.of(true); + return ConstantMatcherType.ALL_TRUE.asValueMatcher(); } @Override public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) { - return BooleanVectorValueMatcher.of(factory.getReadableVectorInspector(), true); + return ConstantMatcherType.ALL_TRUE.asVectorMatcher(factory.getReadableVectorInspector()); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java b/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java index 75b4dcc77f4..9b4ae9e5868 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ValueMatchers.java @@ -31,9 +31,12 @@ import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.BaseNullableColumnValueSelector; +import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.DimensionDictionarySelector; import org.apache.druid.segment.DimensionHandlerUtils; import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.IdLookup; +import org.apache.druid.segment.data.IndexedInts; import javax.annotation.Nullable; import java.util.Objects; @@ -49,6 +52,31 @@ public class ValueMatchers // No instantiation. } + /** + * Matcher for constant 'true' condition, where all rows should always match + */ + public static ValueMatcher allTrue() + { + return AllTrueValueMatcher.instance(); + } + + /** + * Matcher for constant 'false' condition, where rows will never be matched + */ + public static ValueMatcher allFalse() + { + return AllFalseValueMatcher.instance(); + } + + /** + * Matcher for constant 'unknown' condition, such as a column of all null values, where rows will never match + * unless {@code includeUnknown} is set to true on the match function. + */ + public static ValueMatcher allUnknown() + { + return AllUnknownValueMatcher.instance(); + } + /** * Creates a constant-based {@link ValueMatcher} for a string-typed selector. * @@ -62,17 +90,16 @@ public class ValueMatchers final boolean hasMultipleValues ) { - final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible( + final ConstantMatcherType matcherType = toConstantMatcherTypeIfPossible( selector, hasMultipleValues, s -> Objects.equals(s, NullHandling.emptyToNullIfNeeded(value)) ); - - if (booleanMatcher != null) { - return booleanMatcher; - } else { - return selector.makeValueMatcher(value); + if (matcherType != null) { + return matcherType.asValueMatcher(); } + return selector.makeValueMatcher(value); + } /** @@ -89,12 +116,16 @@ public class ValueMatchers ) { final Predicate predicate = predicateFactory.makeStringPredicate(); - final ValueMatcher booleanMatcher = toBooleanMatcherIfPossible(selector, hasMultipleValues, predicate); + final ConstantMatcherType constantMatcherType = toConstantMatcherTypeIfPossible( + selector, + hasMultipleValues, + predicate + ); - if (booleanMatcher != null) { - return booleanMatcher; + if (constantMatcherType != null) { + return constantMatcherType.asValueMatcher(); } else { - return selector.makeValueMatcher(predicate); + return selector.makeValueMatcher(predicateFactory); } } @@ -133,10 +164,10 @@ public class ValueMatchers return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { if (selector.isNull()) { - return false; + return includeUnknown; } return Float.floatToIntBits(selector.getFloat()) == matchValIntBits; } @@ -149,63 +180,6 @@ public class ValueMatchers }; } - public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, final String value) - { - final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value); - if (matchVal == null) { - return makeNumericNullValueMatcher(selector); - } - return makeLongValueMatcher(selector, matchVal); - } - - public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, long value) - { - return new ValueMatcher() - { - @Override - public boolean matches() - { - if (selector.isNull()) { - return false; - } - return selector.getLong() == value; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("selector", selector); - } - }; - } - - public static ValueMatcher makeLongValueMatcher( - final BaseLongColumnValueSelector selector, - final DruidPredicateFactory predicateFactory - ) - { - final DruidLongPredicate predicate = predicateFactory.makeLongPredicate(); - return new ValueMatcher() - { - @Override - public boolean matches() - { - if (selector.isNull()) { - return predicate.applyNull(); - } - return predicate.applyLong(selector.getLong()); - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("selector", selector); - inspector.visit("predicate", predicate); - } - }; - } - - /** * Creates a predicate-based {@link ValueMatcher} for a float-typed selector. * @@ -221,10 +195,11 @@ public class ValueMatchers return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); if (selector.isNull()) { - return predicate.applyNull(); + return matchNull || predicate.applyNull(); } return predicate.applyFloat(selector.getFloat()); } @@ -238,6 +213,82 @@ public class ValueMatchers }; } + /** + * Creates a constant-based {@link ValueMatcher} for a long-typed selector. + * + * @param selector column selector + * @param value value to match + */ + public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, final String value) + { + final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value); + if (matchVal == null) { + return makeNumericNullValueMatcher(selector); + } + return makeLongValueMatcher(selector, matchVal); + } + + /** + * Creates a constant-based {@link ValueMatcher} for a long-typed selector. + * + * @param selector column selector + * @param value value to match + */ + public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, long value) + { + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + if (selector.isNull()) { + return includeUnknown; + } + return selector.getLong() == value; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } + + /** + * Creates a predicate-based {@link ValueMatcher} for a long-typed selector. + * + * @param selector column selector + * @param predicateFactory predicate to match + */ + public static ValueMatcher makeLongValueMatcher( + final BaseLongColumnValueSelector selector, + final DruidPredicateFactory predicateFactory + ) + { + final DruidLongPredicate predicate = predicateFactory.makeLongPredicate(); + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + if (selector.isNull()) { + return matchNull || predicate.applyNull(); + } + return predicate.applyLong(selector.getLong()); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + inspector.visit("predicate", predicate); + } + }; + } + + /** * Creates a constant-based {@link ValueMatcher} for a double-typed selector. * @@ -257,6 +308,12 @@ public class ValueMatchers return makeDoubleValueMatcher(selector, matchVal); } + /** + * Creates a constant-based {@link ValueMatcher} for a double-typed selector. + * + * @param selector column selector + * @param value value to match + */ public static ValueMatcher makeDoubleValueMatcher( final BaseDoubleColumnValueSelector selector, final double value @@ -267,10 +324,10 @@ public class ValueMatchers return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { if (selector.isNull()) { - return false; + return includeUnknown; } return Double.doubleToLongBits(selector.getDouble()) == matchValLongBits; } @@ -298,10 +355,11 @@ public class ValueMatchers return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); if (selector.isNull()) { - return predicate.applyNull(); + return matchNull || predicate.applyNull(); } return predicate.applyDouble(selector.getDouble()); } @@ -315,6 +373,140 @@ public class ValueMatchers }; } + /** + * Create a matcher that should always return false, except when {@code includeUnknown} is set, in which case only + * null values will be matched. This is typically used when the filter should never match any actual values, but + * still needs to be able to report 'unknown' matches. + */ + public static ValueMatcher makeAlwaysFalseDimensionMatcher(final DimensionSelector selector, boolean multiValue) + { + final IdLookup lookup = selector.idLookup(); + // if the column doesn't have null + if (lookup == null || !selector.nameLookupPossibleInAdvance()) { + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + if (includeUnknown) { + IndexedInts row = selector.getRow(); + final int size = row.size(); + if (size == 0) { + return true; + } + for (int i = 0; i < size; i++) { + if (NullHandling.isNullOrEquivalent(selector.lookupName(row.get(i)))) { + return true; + } + } + } + return false; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } else { + final int nullId = lookup.lookupId(null); + if (nullId < 0) { + // column doesn't have null value so no unknowns, can safely return always false matcher + return ValueMatchers.allFalse(); + } + if (multiValue) { + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + if (includeUnknown) { + IndexedInts row = selector.getRow(); + final int size = row.size(); + if (size == 0) { + return true; + } + for (int i = 0; i < size; i++) { + if (row.get(i) == nullId) { + return true; + } + } + } + return false; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } else { + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + return includeUnknown && selector.getRow().get(0) == nullId; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } + } + } + + /** + * Create a matcher that should always return false, except when {@code includeUnknown} is set, in which case only + * null values will be matched. This is typically used when the filter should never match any actual values, but + * still needs to be able to report 'unknown' matches. + */ + public static ValueMatcher makeAlwaysFalseNumericMatcher(BaseNullableColumnValueSelector selector) + { + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + return includeUnknown && selector.isNull(); + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } + + /** + * Create a matcher that should always return false, except when {@code includeUnknown} is set, in which case only + * null values will be matched. This is typically used when the filter should never match any actual values, but + * still needs to be able to report 'unknown' matches. + */ + public static ValueMatcher makeAlwaysFalseObjectMatcher(BaseObjectColumnValueSelector selector) + { + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + return includeUnknown && selector.getObject() == null; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("selector", selector); + } + }; + } + /** * If applying {@code predicate} to {@code selector} would always return a constant, returns that constant. * Otherwise, returns null. @@ -327,7 +519,7 @@ public class ValueMatchers * @param predicate predicate to apply */ @Nullable - public static Boolean toBooleanIfPossible( + public static ConstantMatcherType toConstantMatcherTypeIfPossible( final DimensionDictionarySelector selector, final boolean hasMultipleValues, final Predicate predicate @@ -336,36 +528,22 @@ public class ValueMatchers if (selector.getValueCardinality() == 0) { // Column has no values (it doesn't exist, or it's all empty arrays). // Match if and only if "predicate" matches null. - return predicate.apply(null); + if (predicate.apply(null)) { + return ConstantMatcherType.ALL_TRUE; + } + return ConstantMatcherType.ALL_UNKNOWN; } else if (!hasMultipleValues && selector.getValueCardinality() == 1 && selector.nameLookupPossibleInAdvance()) { // Every row has the same value. Match if and only if "predicate" matches the possible value. - return predicate.apply(selector.lookupName(0)); - } else { - return null; + final String constant = selector.lookupName(0); + if (predicate.apply(constant)) { + return ConstantMatcherType.ALL_TRUE; + } + if (constant == null) { + return ConstantMatcherType.ALL_UNKNOWN; + } + return ConstantMatcherType.ALL_FALSE; } - } - - /** - * If {@link #toBooleanIfPossible} would return nonnull, this returns a {@link BooleanValueMatcher} that always - * returns that value. Otherwise, this returns null. - * - * @param selector string selector - * @param hasMultipleValues whether the selector *might* have multiple values - * @param predicate predicate to apply - */ - @Nullable - private static ValueMatcher toBooleanMatcherIfPossible( - final DimensionSelector selector, - final boolean hasMultipleValues, - final Predicate predicate - ) - { - final Boolean booleanValue = ValueMatchers.toBooleanIfPossible( - selector, - hasMultipleValues, - predicate - ); - return booleanValue == null ? null : BooleanValueMatcher.of(booleanValue); + return null; } /** @@ -377,7 +555,7 @@ public class ValueMatchers return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { return selector.isNull(); } diff --git a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java index 565ec3294fb..8d72133a205 100644 --- a/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/incremental/IncrementalIndexStorageAdapter.java @@ -43,7 +43,7 @@ import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.data.ListIndexed; -import org.apache.druid.segment.filter.BooleanValueMatcher; +import org.apache.druid.segment.filter.ValueMatchers; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -326,7 +326,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter ); // Set maxRowIndex before creating the filterMatcher. See https://github.com/apache/druid/pull/6340 maxRowIndex = index.getLastRowIndex(); - filterMatcher = filter == null ? BooleanValueMatcher.of(true) : filter.makeMatcher(columnSelectorFactory); + filterMatcher = filter == null ? ValueMatchers.allTrue() : filter.makeMatcher(columnSelectorFactory); numAdvanced = -1; final long timeStart = Math.max(interval.getStartMillis(), actualInterval.getStartMillis()); cursorIterable = index.getFacts().timeRangeIterable( @@ -370,7 +370,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter currEntry.set(entry); - if (filterMatcher.matches()) { + if (filterMatcher.matches(false)) { return; } } @@ -398,7 +398,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter currEntry.set(entry); - if (filterMatcher.matches()) { + if (filterMatcher.matches(false)) { return; } } @@ -439,7 +439,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter continue; } currEntry.set(entry); - if (filterMatcher.matches()) { + if (filterMatcher.matches(false)) { foundMatched = true; break; } diff --git a/processing/src/main/java/org/apache/druid/segment/index/AllFalseBitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/AllFalseBitmapColumnIndex.java index a7518d44c6b..ba46e39da2a 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/AllFalseBitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/AllFalseBitmapColumnIndex.java @@ -20,23 +20,28 @@ package org.apache.druid.segment.index; import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.BitmapResultFactory; -import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; +import javax.annotation.Nullable; + public class AllFalseBitmapColumnIndex implements BitmapColumnIndex { private final BitmapFactory bitmapFactory; - - public AllFalseBitmapColumnIndex(ColumnIndexSelector indexSelector) - { - this(indexSelector.getBitmapFactory()); - } + @Nullable + private final ImmutableBitmap unknownBitmap; public AllFalseBitmapColumnIndex(BitmapFactory bitmapFactory) + { + this(bitmapFactory, null); + } + + public AllFalseBitmapColumnIndex(BitmapFactory bitmapFactory, @Nullable ImmutableBitmap unknownBitmap) { this.bitmapFactory = bitmapFactory; + this.unknownBitmap = unknownBitmap; } @Override @@ -52,8 +57,11 @@ public class AllFalseBitmapColumnIndex implements BitmapColumnIndex } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { + if (includeUnknown && unknownBitmap != null) { + return bitmapResultFactory.wrapDimensionValue(unknownBitmap); + } return bitmapResultFactory.wrapAllFalse(bitmapFactory.makeEmptyImmutableBitmap()); } } diff --git a/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java index 686add0693c..db3a0293cfa 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java @@ -46,7 +46,7 @@ public class AllTrueBitmapColumnIndex implements BitmapColumnIndex } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { return bitmapResultFactory.wrapAllTrue( selector.getBitmapFactory() diff --git a/processing/src/main/java/org/apache/druid/segment/index/AllUnknownBitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/AllUnknownBitmapColumnIndex.java new file mode 100644 index 00000000000..b247face713 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/index/AllUnknownBitmapColumnIndex.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.segment.index; + +import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.query.filter.ColumnIndexSelector; +import org.apache.druid.segment.column.ColumnIndexCapabilities; +import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; + +/** + * Like {@link AllFalseBitmapColumnIndex} during normal operation, except if called with 'includeUnknowns' becomes like + * {@link AllTrueBitmapColumnIndex}. + */ +public class AllUnknownBitmapColumnIndex implements BitmapColumnIndex +{ + private final ColumnIndexSelector selector; + + + public AllUnknownBitmapColumnIndex(ColumnIndexSelector indexSelector) + { + this.selector = indexSelector; + } + + @Override + public ColumnIndexCapabilities getIndexCapabilities() + { + return SimpleColumnIndexCapabilities.getConstant(); + } + + @Override + public double estimateSelectivity(int totalRows) + { + return 0; + } + + @Override + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) + { + if (includeUnknown) { + return bitmapResultFactory.wrapAllTrue( + selector.getBitmapFactory() + .complement(selector.getBitmapFactory().makeEmptyImmutableBitmap(), selector.getNumRows()) + ); + } + return bitmapResultFactory.wrapAllFalse(selector.getBitmapFactory().makeEmptyImmutableBitmap()); + } +} 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 77b78ac0698..23170154335 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 @@ -22,11 +22,27 @@ package org.apache.druid.segment.index; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.segment.column.ColumnIndexCapabilities; +/** + * 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. + */ public interface BitmapColumnIndex { ColumnIndexCapabilities getIndexCapabilities(); double estimateSelectivity(int totalRows); - T computeBitmapResult(BitmapResultFactory bitmapResultFactory); + /** + * Compute a bitmap result wrapped with the {@link BitmapResultFactory} representing the rows matched by this index. + * + * @param bitmapResultFactory helper to format the {@link org.apache.druid.collections.bitmap.ImmutableBitmap} in a + * form ready for consumption by callers + * @param includeUnknown mapping for Druid native two state logic system into SQL three-state logic system. If set + * to true, this method should also return true if the result is 'unknown' to be a match, + * such as from the input being null valued. Used primarily to allow + * {@link org.apache.druid.segment.filter.NotFilter} to invert a match in an SQL compliant + * manner + * @return bitmap result representing rows matched by this index + */ + T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown); } diff --git a/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java index 9038ed560e1..43eb987a27a 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.index; import com.google.common.base.Predicate; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; @@ -116,6 +117,16 @@ public final class IndexedStringDruidPredicateIndexes matcher ) { + final IntIntPair range = getRange(startValue, startStrict, endValue, endStrict); + final int start = range.leftInt(), end = range.rightInt(); + if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, end - start)) { + return null; + } return new SimpleImmutableBitmapIterableIndex() { @Override public Iterable getBitmapIterable() { - final IntIntPair range = getRange(startValue, startStrict, endValue, endStrict); - final int start = range.leftInt(), end = range.rightInt(); return () -> new Iterator() { int currIndex = start; @@ -169,6 +182,16 @@ public final class IndexedUtf8LexicographicalRangeIndexes T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { - + if (includeUnknown && NullHandling.isNullOrEquivalent(dictionary.get(0))) { + return bitmapResultFactory.unionDimensionValueBitmaps( + ImmutableList.of(getBitmapForValue(), getBitmap(0)) + ); + } return bitmapResultFactory.wrapDimensionValue(getBitmapForValue()); } @@ -114,6 +121,7 @@ public final class IndexedUtf8ValueIndexes values) { @@ -122,10 +130,12 @@ public final class IndexedUtf8ValueIndexes valuesUtf8) { @@ -138,7 +148,7 @@ public final class IndexedUtf8ValueIndexes valuesUtf8, int size) + 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()) { @@ -213,19 +223,29 @@ public final class IndexedUtf8ValueIndexes valuesUtf8) + private SimpleImmutableBitmapIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable valuesUtf8, boolean valuesContainsNull) { return new SimpleImmutableBitmapIterableIndex() { @@ -277,6 +297,16 @@ public final class IndexedUtf8ValueIndexes T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { return bitmapResultFactory.wrapDimensionValue(bitmap); } diff --git a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java index 2a6d47c1b53..99bc1d5702e 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java @@ -19,10 +19,14 @@ package org.apache.druid.segment.index; +import com.google.common.collect.Iterables; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.segment.filter.Filters; +import javax.annotation.Nullable; +import java.util.Collections; + /** * {@link SimpleBitmapColumnIndex} for anything which can compute an {@link Iterable} in some manner */ @@ -35,10 +39,24 @@ public abstract class SimpleImmutableBitmapIterableIndex extends SimpleBitmapCol } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { + if (includeUnknown) { + final ImmutableBitmap unknownsBitmap = getUnknownsBitmap(); + if (unknownsBitmap != null) { + return bitmapResultFactory.unionDimensionValueBitmaps( + Iterables.concat( + getBitmapIterable(), + Collections.singletonList(unknownsBitmap) + ) + ); + } + } return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable()); } protected abstract Iterable getBitmapIterable(); + + @Nullable + protected abstract ImmutableBitmap getUnknownsBitmap(); } diff --git a/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullDimensionSelector.java index ab3fb87a296..7d65075429b 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PossiblyNullDimensionSelector.java @@ -19,7 +19,7 @@ package org.apache.druid.segment.join; -import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; @@ -80,9 +80,9 @@ public class PossiblyNullDimensionSelector extends AbstractDimensionSelector imp @Override @Nonnull - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java index 11370fc167b..57da128c73d 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java +++ b/processing/src/main/java/org/apache/druid/segment/join/PostJoinCursor.java @@ -72,7 +72,7 @@ public class PostJoinCursor implements Cursor private void advanceToMatch() { if (valueMatcher != null) { - while (!isDone() && !valueMatcher.matches()) { + while (!isDone() && !valueMatcher.matches(false)) { baseCursor.advanceUninterruptibly(); } } diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableDimensionSelector.java index 6848679e2cf..9eb9fd02f5f 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/IndexedTableDimensionSelector.java @@ -19,9 +19,9 @@ package org.apache.druid.segment.join.table; -import com.google.common.base.Predicate; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.DimensionHandlerUtils; @@ -81,9 +81,9 @@ public class IndexedTableDimensionSelector implements DimensionSelector } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java index 64c062bf417..2fb02701f75 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplier.java @@ -22,6 +22,7 @@ package org.apache.druid.segment.nested; import com.google.common.base.Predicate; import com.google.common.base.Strings; import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; import com.google.common.primitives.Doubles; import it.unimi.dsi.fastutil.doubles.DoubleArraySet; import it.unimi.dsi.fastutil.doubles.DoubleIterator; @@ -34,6 +35,7 @@ import it.unimi.dsi.fastutil.ints.IntList; import it.unimi.dsi.fastutil.longs.LongArraySet; import it.unimi.dsi.fastutil.longs.LongIterator; import it.unimi.dsi.fastutil.longs.LongSet; +import org.apache.druid.annotations.SuppressFBWarnings; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.common.config.NullHandling; @@ -310,6 +312,16 @@ public class NestedFieldColumnIndexSupplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { final int globalId = stringDictionary.indexOf(StringUtils.toUtf8ByteBuffer(value)); + if (includeUnknown && localDictionary.get(0) == 0) { + if (globalId < 0) { + return bitmapResultFactory.wrapDimensionValue(bitmaps.get(0)); + } + return bitmapResultFactory.unionDimensionValueBitmaps( + ImmutableList.of( + getBitmap(localDictionary.indexOf(globalId + adjustDoubleId)), + bitmaps.get(0) + ) + ); + } if (globalId < 0) { return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); } @@ -431,6 +454,17 @@ public class NestedFieldColumnIndexSupplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { if (longValue == null) { if (inputNull) { @@ -638,6 +692,17 @@ public class NestedFieldColumnIndexSupplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { if (doubleValue == null) { if (inputNull) { @@ -866,6 +952,17 @@ public class NestedFieldColumnIndexSupplier predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { final BitSet checkedIds = new BitSet(getCardinality()); final BitSet matchingIds = new BitSet(getCardinality()); + final Predicate predicate = predicateFactory.makeStringPredicate(); // Lazy matcher; only check an id if matches() is called. return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final int id = getRowValue(); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); if (checkedIds.get(id)) { return matchingIds.get(id); } else { - final boolean matches = predicate.apply(lookupName(id)); + final String rowVal = lookupName(id); + final boolean matches = (matchNull && rowVal == null) || predicate.apply(rowVal); checkedIds.set(id); if (matches) { matchingIds.set(id); 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 ef1f6ac5299..b5a31448302 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 @@ -28,6 +28,7 @@ import it.unimi.dsi.fastutil.doubles.DoubleIterator; import it.unimi.dsi.fastutil.doubles.DoubleSet; import it.unimi.dsi.fastutil.ints.IntIntPair; import it.unimi.dsi.fastutil.ints.IntIterator; +import org.apache.druid.annotations.SuppressFBWarnings; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.common.config.NullHandling; @@ -235,7 +236,7 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.DOUBLE); if (castForComparison == null) { - return new AllFalseBitmapColumnIndex(bitmapFactory); + return new AllFalseBitmapColumnIndex(bitmapFactory, nullValueBitmap); } final double doubleValue = castForComparison.asDouble(); return new SimpleBitmapColumnIndex() @@ -252,9 +253,17 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { final int id = dictionary.indexOf(doubleValue); + if (includeUnknown) { + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(nullValueBitmap); + } + return bitmapResultFactory.unionDimensionValueBitmaps( + ImmutableList.of(getBitmap(id), nullValueBitmap) + ); + } if (id < 0) { return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); } @@ -300,7 +309,7 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { if (doubleValue == null) { if (inputNull && NullHandling.sqlCompatible()) { @@ -322,6 +331,14 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier eval = ExprEval.ofType(ExpressionType.fromColumnTypeStrict(valueType), value); final ExprEval castForComparison = ExprEval.castForEqualityComparison(eval, ExpressionType.LONG); if (castForComparison == null) { - return new AllFalseBitmapColumnIndex(bitmapFactory); + return new AllFalseBitmapColumnIndex(bitmapFactory, nullValueBitmap); } final long longValue = castForComparison.asLong(); @@ -253,9 +254,17 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { final int id = dictionary.indexOf(longValue); + if (includeUnknown) { + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(nullValueBitmap); + } + return bitmapResultFactory.unionDimensionValueBitmaps( + ImmutableList.of(getBitmap(id), nullValueBitmap) + ); + } if (id < 0) { return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); } @@ -301,7 +310,7 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { if (longValue == null) { if (inputNull && NullHandling.sqlCompatible()) { @@ -322,6 +331,14 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier> return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { - return valueIds.contains(getRowValue()); + final int rowId = getRowValue(); + // null is always 0 + return (includeUnknown && rowId == 0) || valueIds.contains(getRowValue()); } @Override @@ -489,32 +491,50 @@ public class VariantColumn> } }; } else { - return BooleanValueMatcher.of(false); + return new ValueMatcher() + { + @Override + public boolean matches(boolean includeUnknown) + { + // null is always 0 + return includeUnknown && getRowValue() == 0; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("column", VariantColumn.this); + } + }; } } else { // Employ caching BitSet optimization - return makeValueMatcher(Predicates.equalTo(value)); + return makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(value)); } } @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { final BitSet checkedIds = new BitSet(getCardinality()); final BitSet matchingIds = new BitSet(getCardinality()); + final Predicate predicate = predicateFactory.makeStringPredicate(); // Lazy matcher; only check an id if matches() is called. return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { final int id = getRowValue(); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + if (checkedIds.get(id)) { return matchingIds.get(id); } else { - final boolean matches = predicate.apply(lookupName(id)); + final String val = lookupName(id); + final boolean matches = (matchNull && val == null) || predicate.apply(lookupName(id)); checkedIds.set(id); if (matches) { matchingIds.set(id); diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java index afe8b630e3c..15d74e940bd 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumnAndIndexSupplier.java @@ -20,6 +20,7 @@ package org.apache.druid.segment.nested; import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.error.DruidException; @@ -325,7 +326,7 @@ public class VariantColumnAndIndexSupplier implements Supplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { final int id = dictionary.indexOf(ids) + arrayOffset; + if (includeUnknown) { + if (id < 0) { + return bitmapResultFactory.wrapDimensionValue(nullValueBitmap); + } + return bitmapResultFactory.unionDimensionValueBitmaps( + ImmutableList.of(getBitmap(id), nullValueBitmap) + ); + } if (id < 0) { return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); } @@ -406,7 +415,7 @@ public class VariantColumnAndIndexSupplier implements Supplier T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { final int elementId = getElementId(); - + if (includeUnknown) { + if (elementId < 0) { + return bitmapResultFactory.wrapDimensionValue(nullValueBitmap); + } + return bitmapResultFactory.unionDimensionValueBitmaps( + ImmutableList.of(getElementBitmap(elementId), nullValueBitmap) + ); + } if (elementId < 0) { return bitmapResultFactory.wrapDimensionValue(bitmapFactory.makeEmptyImmutableBitmap()); } diff --git a/processing/src/main/java/org/apache/druid/segment/selector/settable/SettableDimensionValueSelector.java b/processing/src/main/java/org/apache/druid/segment/selector/settable/SettableDimensionValueSelector.java index f21f9f289bf..0d362c50411 100644 --- a/processing/src/main/java/org/apache/druid/segment/selector/settable/SettableDimensionValueSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/selector/settable/SettableDimensionValueSelector.java @@ -19,7 +19,7 @@ package org.apache.druid.segment.selector.settable; -import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; @@ -66,7 +66,7 @@ public class SettableDimensionValueSelector implements DimensionSelector, Settab } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { throw new UnsupportedOperationException(); } diff --git a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java index d199a45bbf5..65acf7c58d0 100644 --- a/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java +++ b/processing/src/main/java/org/apache/druid/segment/transform/Transformer.java @@ -92,7 +92,7 @@ public class Transformer if (valueMatcher != null) { rowSupplierForValueMatcher.set(transformedRow); - if (!valueMatcher.matches()) { + if (!valueMatcher.matches(false)) { return null; } } @@ -136,7 +136,7 @@ public class Transformer final List> inputVals = inputRowListPlusRawValues.getRawValuesList(); for (int i = 0; i < size; i++) { rowSupplierForValueMatcher.set(inputRows.get(i)); - if (valueMatcher.matches()) { + if (valueMatcher.matches(false)) { matchedRows.add(inputRows.get(i)); matchedVals.add(inputVals.get(i)); } diff --git a/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java b/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java index 4ced41acb0c..c6da18edb8b 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java @@ -129,7 +129,10 @@ public class FilteredVectorOffset implements VectorOffset return; } - final ReadableVectorMatch match = filterMatcher.match(VectorMatch.allTrue(baseOffset.getCurrentVectorSize())); + final ReadableVectorMatch match = filterMatcher.match( + VectorMatch.allTrue(baseOffset.getCurrentVectorSize()), + false + ); if (match.isAllTrue(baseOffset.getCurrentVectorSize())) { currentVectorSize = baseOffset.getCurrentVectorSize(); diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionMultiValueDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionMultiValueDimensionSelector.java index 56e33088deb..6ede3ae1c70 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionMultiValueDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionMultiValueDimensionSelector.java @@ -24,6 +24,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.math.expr.Evals; import org.apache.druid.math.expr.ExprEval; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; @@ -127,14 +128,15 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { ExprEval evaluated = getEvaluated(); if (evaluated.isArray()) { List array = getArrayAsList(evaluated); - return array.stream().anyMatch(x -> Objects.equals(x, value)); + return array.stream().anyMatch(x -> (includeUnknown && x == null) || Objects.equals(x, value)); } - return Objects.equals(getValue(evaluated), value); + final String rowValue = getValue(evaluated); + return (includeUnknown && rowValue == null) || Objects.equals(rowValue, value); } @Override @@ -146,26 +148,29 @@ public class ExpressionMultiValueDimensionSelector implements DimensionSelector } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { return new ValueMatcher() { @Override - public boolean matches() + public boolean matches(boolean includeUnknown) { ExprEval evaluated = getEvaluated(); + final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown(); + final Predicate predicate = predicateFactory.makeStringPredicate(); if (evaluated.isArray()) { List array = getArrayAsList(evaluated); - return array.stream().anyMatch(x -> predicate.apply(x)); + return array.stream().anyMatch(x -> (matchNull && x == null) || predicate.apply(x)); } - return predicate.apply(getValue(evaluated)); + final String rowValue = getValue(evaluated); + return (matchNull && rowValue == null) || predicate.apply(rowValue); } @Override public void inspectRuntimeShape(RuntimeShapeInspector inspector) { inspector.visit("selector", baseSelector); - inspector.visit("predicate", predicate); + inspector.visit("predicate", predicateFactory); } }; } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java index dedc5a241d7..6597472b47b 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ExpressionSelectors.java @@ -435,17 +435,17 @@ public class ExpressionSelectors if (row.size() == 1 && !coerceArray) { return selector.lookupName(row.get(0)); } else { + final int size = row.size(); // column selector factories hate you and use [] and [null] interchangeably for nullish data - if (row.size() == 0 || (row.size() == 1 && selector.getObject() == null)) { + if (size == 0 || (size == 1 && selector.getObject() == null)) { if (homogenize) { return new Object[]{null}; } else { return null; } } - final Object[] strings = new Object[row.size()]; - // noinspection SSBasedInspection - for (int i = 0; i < row.size(); i++) { + final Object[] strings = new Object[size]; + for (int i = 0; i < size; i++) { strings[i] = selector.lookupName(row.get(i)); } return strings; diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java index 0ffabc8cc34..fa0aea6778a 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/ListFilteredVirtualColumn.java @@ -24,6 +24,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; import com.google.common.base.Predicate; import com.google.common.base.Predicates; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.common.config.NullHandling; @@ -197,9 +201,6 @@ public class ListFilteredVirtualColumn implements VirtualColumn if (holder == null) { return null; } - // someday maybe we can have a better way to get row count.. - final ColumnHolder time = columnSelector.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME); - final int numRows = time.getLength(); ColumnIndexSupplier indexSupplier = holder.getIndexSupplier(); if (indexSupplier == null) { @@ -220,6 +221,7 @@ public class ListFilteredVirtualColumn implements VirtualColumn null, underlyingIndex::getValue ); + } else { idMapping = ListFilteredDimensionSpec.buildDenyListIdMapping( values, @@ -228,14 +230,24 @@ public class ListFilteredVirtualColumn implements VirtualColumn ); } + // someday maybe we can have a better way to get row count.. + final ColumnHolder time = columnSelector.getColumnHolder(ColumnHolder.TIME_COLUMN_NAME); + final int numRows = time.getLength(); + final Supplier nullValueBitmapSupplier = Suppliers.memoize( + () -> underlyingIndex.getBitmapFactory().complement( + underlyingIndex.getBitmapFactory().union(getNonNullBitmaps(underlyingIndex, idMapping)), + numRows + ) + ); + if (clazz.equals(NullValueIndex.class)) { - return (T) new ListFilteredNullValueIndex(underlyingIndex, idMapping, numRows); + return (T) new ListFilteredNullValueIndex(nullValueBitmapSupplier); } else if (clazz.equals(StringValueSetIndexes.class)) { - return (T) new ListFilteredStringValueSetIndexes(underlyingIndex, idMapping); + return (T) new ListFilteredStringValueSetIndexes(underlyingIndex, idMapping, nullValueBitmapSupplier); } else if (clazz.equals(DruidPredicateIndexes.class)) { - return (T) new ListFilteredDruidPredicateIndexes(underlyingIndex, idMapping); + return (T) new ListFilteredDruidPredicateIndexes(underlyingIndex, idMapping, nullValueBitmapSupplier); } else if (clazz.equals(LexicographicalRangeIndexes.class)) { - return (T) new ListFilteredLexicographicalRangeIndexes(underlyingIndex, idMapping); + return (T) new ListFilteredLexicographicalRangeIndexes(underlyingIndex, idMapping, nullValueBitmapSupplier); } else if (clazz.equals(DictionaryEncodedStringValueIndex.class) || clazz.equals(DictionaryEncodedValueIndex.class)) { return (T) new ListFilteredDictionaryEncodedStringValueIndex(underlyingIndex, idMapping); } @@ -275,6 +287,66 @@ public class ListFilteredVirtualColumn implements VirtualColumn '}'; } + private static Iterable getNonNullBitmaps( + DictionaryEncodedStringValueIndex delegateIndex, + IdMapping idMapping + ) + { + final int start = NullHandling.isNullOrEquivalent(delegateIndex.getValue(idMapping.getReverseId(0))) ? 1 : 0; + return getBitmapsInRange(delegateIndex, idMapping, v -> true, start, idMapping.getValueCardinality()); + } + + private static Iterable getBitmapsInRange( + DictionaryEncodedStringValueIndex delegate, + IdMapping idMapping, + Predicate matcher, + int start, + int end + ) + { + return () -> new Iterator() + { + int currIndex = start; + int found; + + { + found = findNext(); + } + + private int findNext() + { + while (currIndex < end && !matcher.apply(delegate.getValue(idMapping.getReverseId(currIndex)))) { + currIndex++; + } + + if (currIndex < end) { + return currIndex++; + } else { + return -1; + } + } + + @Override + public boolean hasNext() + { + return found != -1; + } + + @Override + public ImmutableBitmap next() + { + int cur = found; + + if (cur == -1) { + throw new NoSuchElementException(); + } + + found = findNext(); + return delegate.getBitmap(idMapping.getReverseId(cur)); + } + }; + } + private static class BaseListFilteredColumnIndex { final DictionaryEncodedStringValueIndex delegate; @@ -325,85 +397,37 @@ public class ListFilteredVirtualColumn implements VirtualColumn Iterable getBitmapsInRange(Predicate matcher, int start, int end) { - return () -> new Iterator() - { - int currIndex = start; - int found; - - { - found = findNext(); - } - - private int findNext() - { - while (currIndex < end && !matcher.apply(delegate.getValue(idMapping.getReverseId(currIndex)))) { - currIndex++; - } - - if (currIndex < end) { - return currIndex++; - } else { - return -1; - } - } - - @Override - public boolean hasNext() - { - return found != -1; - } - - @Override - public ImmutableBitmap next() - { - int cur = found; - - if (cur == -1) { - throw new NoSuchElementException(); - } - - found = findNext(); - return getBitmap(cur); - } - }; + return ListFilteredVirtualColumn.getBitmapsInRange(delegate, idMapping, matcher, start, end); } } - private static class ListFilteredNullValueIndex extends BaseListFilteredColumnIndex implements NullValueIndex + private static class ListFilteredNullValueIndex implements NullValueIndex { - private final int numRows; + private final Supplier nullValueBitmapSupplier; - private ListFilteredNullValueIndex(DictionaryEncodedStringValueIndex delegate, IdMapping idMapping, int numRows) + private ListFilteredNullValueIndex(Supplier nullValueBitmapSupplier) { - super(delegate, idMapping); - this.numRows = numRows; + this.nullValueBitmapSupplier = nullValueBitmapSupplier; } @Override public BitmapColumnIndex get() { - return new SimpleImmutableBitmapIterableIndex() + return new SimpleBitmapColumnIndex() { @Override public double estimateSelectivity(int totalRows) { - return 1.0 - Filters.estimateSelectivity(getBitmapIterable().iterator(), totalRows); - } - - @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) - { - return bitmapResultFactory.complement( - bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable()), - numRows + return Filters.estimateSelectivity( + Collections.singletonList(nullValueBitmapSupplier.get()).iterator(), + totalRows ); } @Override - protected Iterable getBitmapIterable() + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknowns) { - final int start = NullHandling.isNullOrEquivalent(delegate.getValue(idMapping.getReverseId(0))) ? 1 : 0; - return getBitmapsInRange(v -> true, start, idMapping.getValueCardinality()); + return bitmapResultFactory.wrapDimensionValue(nullValueBitmapSupplier.get()); } }; } @@ -412,13 +436,16 @@ public class ListFilteredVirtualColumn implements VirtualColumn private static class ListFilteredStringValueSetIndexes extends BaseListFilteredColumnIndex implements StringValueSetIndexes { + private final Supplier nullValueBitmapSupplier; private ListFilteredStringValueSetIndexes( DictionaryEncodedStringValueIndex delegate, - IdMapping idMapping + IdMapping idMapping, + Supplier nullValueBitmapSupplier ) { super(delegate, idMapping); + this.nullValueBitmapSupplier = nullValueBitmapSupplier; } @Override @@ -433,8 +460,13 @@ public class ListFilteredVirtualColumn implements VirtualColumn } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { + if (includeUnknown) { + return bitmapResultFactory.unionDimensionValueBitmaps( + ImmutableList.of(getBitmapForValue(), nullValueBitmapSupplier.get()) + ); + } return bitmapResultFactory.wrapDimensionValue(getBitmapForValue()); } @@ -494,6 +526,13 @@ public class ListFilteredVirtualColumn implements VirtualColumn } }; } + + @Nullable + @Override + protected ImmutableBitmap getUnknownsBitmap() + { + return nullValueBitmapSupplier.get(); + } }; } } @@ -501,10 +540,16 @@ public class ListFilteredVirtualColumn implements VirtualColumn private static class ListFilteredDruidPredicateIndexes extends BaseListFilteredColumnIndex implements DruidPredicateIndexes { + private final Supplier nullValueBitmapSupplier; - private ListFilteredDruidPredicateIndexes(DictionaryEncodedStringValueIndex delegate, IdMapping idMapping) + private ListFilteredDruidPredicateIndexes( + DictionaryEncodedStringValueIndex delegate, + IdMapping idMapping, + Supplier nullValueBitmapSupplier + ) { super(delegate, idMapping); + this.nullValueBitmapSupplier = nullValueBitmapSupplier; } @Override @@ -524,12 +569,19 @@ public class ListFilteredVirtualColumn implements VirtualColumn } @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory) + public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { final int start = 0, end = getCardinality(); - return bitmapResultFactory.unionDimensionValueBitmaps( - getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end) - ); + Iterable bitmaps; + if (includeUnknown) { + bitmaps = Iterables.concat( + getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end), + Collections.singletonList(nullValueBitmapSupplier.get()) + ); + } else { + bitmaps = getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end); + } + return bitmapResultFactory.unionDimensionValueBitmaps(bitmaps); } }; } @@ -538,13 +590,16 @@ public class ListFilteredVirtualColumn implements VirtualColumn private static class ListFilteredLexicographicalRangeIndexes extends BaseListFilteredColumnIndex implements LexicographicalRangeIndexes { + private final Supplier nullValueBitmapSupplier; private ListFilteredLexicographicalRangeIndexes( DictionaryEncodedStringValueIndex delegate, - IdMapping idMapping + IdMapping idMapping, + Supplier nullValueBitmapSupplier ) { super(delegate, idMapping); + this.nullValueBitmapSupplier = nullValueBitmapSupplier; } @Override @@ -602,6 +657,13 @@ public class ListFilteredVirtualColumn implements VirtualColumn final int start = startIndex, end = endIndex; return getBitmapsInRange(matcher, start, end); } + + @Nullable + @Override + protected ImmutableBitmap getUnknownsBitmap() + { + return nullValueBitmapSupplier.get(); + } }; } } diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java index 9a93c9428e9..e486e7b7738 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/NestedFieldVirtualColumn.java @@ -24,7 +24,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import com.google.common.primitives.Doubles; import org.apache.druid.common.guava.GuavaUtils; import org.apache.druid.java.util.common.IAE; @@ -35,6 +34,7 @@ import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.cache.CacheKeyBuilder; import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.BaseSingleValueDimensionSelector; @@ -1639,9 +1639,9 @@ public class NestedFieldVirtualColumn implements VirtualColumn } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return baseSelector.makeValueMatcher(predicate); + return baseSelector.makeValueMatcher(predicateFactory); } @Override diff --git a/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionSelector.java b/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionSelector.java index d89033a727e..b8a4e2d82f9 100644 --- a/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionSelector.java +++ b/processing/src/main/java/org/apache/druid/segment/virtual/SingleStringInputDeferredEvaluationExpressionDimensionSelector.java @@ -20,10 +20,10 @@ package org.apache.druid.segment.virtual; import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; import org.apache.druid.java.util.common.ISE; import org.apache.druid.math.expr.Expr; import org.apache.druid.math.expr.ExpressionType; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.DimensionDictionarySelector; @@ -89,9 +89,9 @@ public class SingleStringInputDeferredEvaluationExpressionDimensionSelector impl } @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/test/java/org/apache/druid/extendedset/intset/ImmutableConciseSetTest.java b/processing/src/test/java/org/apache/druid/extendedset/intset/ImmutableConciseSetTest.java index 873b470610f..1107571cf03 100755 --- a/processing/src/test/java/org/apache/druid/extendedset/intset/ImmutableConciseSetTest.java +++ b/processing/src/test/java/org/apache/druid/extendedset/intset/ImmutableConciseSetTest.java @@ -21,6 +21,7 @@ package org.apache.druid.extendedset.intset; import junit.framework.Assert; import org.apache.druid.java.util.common.StringUtils; +import org.junit.Ignore; import org.junit.Test; import java.nio.IntBuffer; @@ -1427,6 +1428,25 @@ public class ImmutableConciseSetTest verifyComplement(expected, testSet, length); } + @Ignore + + @Test + public void testComplement14() + { + // this is a bug in concise implementation if length is 30 or less + List expected = new ArrayList<>(); + int length = 5; + for (int i = 0; i < length; i++) { + expected.add(i); + } + ImmutableConciseSet testSet = ImmutableConciseSet.complement( + ImmutableConciseSet.complement(new ImmutableConciseSet(), length), + length + ); + + verifyComplement(expected, testSet, length); + } + private void verifyComplement(List expected, ImmutableConciseSet set, int endIndex) { List actual = new ArrayList<>(); diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java index 8736c73276c..a18b7cc70ec 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java @@ -23,6 +23,7 @@ import org.apache.datasketches.memory.WritableMemory; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.segment.BaseDoubleColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionDictionarySelector; @@ -130,15 +131,15 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest // Value matcher tests. if (NullHandling.replaceWithDefault()) { - Assert.assertTrue(readSelector.makeValueMatcher("0.0").matches()); - Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches()); - Assert.assertTrue(readSelector.makeValueMatcher("0.0"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher(Objects::isNull).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("0.0").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false)); } else { - Assert.assertFalse(readSelector.makeValueMatcher("0.0").matches()); - Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("0.0"::equals).matches()); - Assert.assertTrue(readSelector.makeValueMatcher(Objects::isNull).matches()); + Assert.assertFalse(readSelector.makeValueMatcher("0.0").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false)); } } @@ -163,10 +164,10 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("5.1").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("5").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("5.1"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("5"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("5.1").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("5").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("5.1")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("5")).matches(false)); } @Test @@ -194,10 +195,10 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("0.5").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("0.5"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("0.5").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("2").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.5")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("2")).matches(false)); } private void writeToMemory(final Double value) diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java index 441858f0fd6..6bae52f1c50 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java @@ -23,6 +23,7 @@ import org.apache.datasketches.memory.WritableMemory; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.segment.BaseFloatColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionDictionarySelector; @@ -130,15 +131,15 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest // Value matcher tests. if (NullHandling.replaceWithDefault()) { - Assert.assertTrue(readSelector.makeValueMatcher("0.0").matches()); - Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches()); - Assert.assertTrue(readSelector.makeValueMatcher("0.0"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher(Objects::isNull).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("0.0").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false)); } else { - Assert.assertFalse(readSelector.makeValueMatcher("0.0").matches()); - Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("0.0"::equals).matches()); - Assert.assertTrue(readSelector.makeValueMatcher(Objects::isNull).matches()); + Assert.assertFalse(readSelector.makeValueMatcher("0.0").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.0")).matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false)); } } @@ -163,10 +164,10 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("5.1").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("5").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("5.1"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("5"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("5.1").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("5").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("5.1")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("5")).matches(false)); } @Test @@ -194,10 +195,10 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("0.5").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("0.5"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("0.5").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("2").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0.5")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("2")).matches(false)); } private void writeToMemory(final Float value) diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java index 7e73a7cfdf0..643846ee0e3 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java @@ -23,6 +23,7 @@ import org.apache.datasketches.memory.WritableMemory; import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.segment.BaseLongColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionDictionarySelector; @@ -130,15 +131,15 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest // Value matcher tests. if (NullHandling.replaceWithDefault()) { - Assert.assertTrue(readSelector.makeValueMatcher("0").matches()); - Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches()); - Assert.assertTrue(readSelector.makeValueMatcher("0"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher(Objects::isNull).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("0").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false)); } else { - Assert.assertFalse(readSelector.makeValueMatcher("0").matches()); - Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("0"::equals).matches()); - Assert.assertTrue(readSelector.makeValueMatcher(Objects::isNull).matches()); + Assert.assertFalse(readSelector.makeValueMatcher("0").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("0")).matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Objects::isNull, false)).matches(false)); } } @@ -163,10 +164,10 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("5").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("5"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("5").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("2").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("5")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("2")).matches(false)); } @Test @@ -194,10 +195,10 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("5").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("5"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("2"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("5").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("2").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("5")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("2")).matches(false)); } private void writeToMemory(final Long value) diff --git a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java index bde3f77480e..a682e658ca7 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java @@ -27,6 +27,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.segment.BaseObjectColumnValueSelector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionDictionarySelector; @@ -232,10 +233,10 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("bar").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("baz").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("bar"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("baz"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("bar").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("baz").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("bar")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("baz")).matches(false)); } @Test @@ -264,10 +265,10 @@ public class StringFieldReaderTest extends InitializedNullHandlingTest Assert.assertNull(readSelector.idLookup()); // Value matcher tests. - Assert.assertTrue(readSelector.makeValueMatcher("ar").matches()); - Assert.assertFalse(readSelector.makeValueMatcher("bar").matches()); - Assert.assertTrue(readSelector.makeValueMatcher("ar"::equals).matches()); - Assert.assertFalse(readSelector.makeValueMatcher("bar"::equals).matches()); + Assert.assertTrue(readSelector.makeValueMatcher("ar").matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher("bar").matches(false)); + Assert.assertTrue(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("ar")).matches(false)); + Assert.assertFalse(readSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("bar")).matches(false)); } private void writeToMemory(@Nullable final List values) diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java index da2e7be83d0..ba86bef8bda 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/FilteredAggregatorTest.java @@ -19,7 +19,6 @@ package org.apache.druid.query.aggregation; -import com.google.common.base.Predicate; import com.google.common.collect.Lists; import org.apache.druid.common.config.NullHandling; import org.apache.druid.js.JavaScriptConfig; @@ -28,6 +27,7 @@ import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.extraction.JavaScriptExtractionFn; import org.apache.druid.query.filter.AndDimFilter; import org.apache.druid.query.filter.BoundDimFilter; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.JavaScriptDimFilter; import org.apache.druid.query.filter.NotDimFilter; @@ -288,9 +288,9 @@ public class FilteredAggregatorTest extends InitializedNullHandlingTest } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java b/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java index 24f4c4eca0b..3a306a6bd9e 100644 --- a/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java +++ b/processing/src/test/java/org/apache/druid/query/aggregation/cardinality/CardinalityAggregatorTest.java @@ -21,7 +21,6 @@ package org.apache.druid.query.aggregation.cardinality; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; -import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; @@ -42,6 +41,7 @@ import org.apache.druid.query.dimension.RegexFilteredDimensionSpec; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.extraction.JavaScriptExtractionFn; import org.apache.druid.query.extraction.RegexDimExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; @@ -147,9 +147,9 @@ public class CardinalityAggregatorTest extends InitializedNullHandlingTest } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/dimension/StringDimensionSelectorForTest.java b/processing/src/test/java/org/apache/druid/query/dimension/StringDimensionSelectorForTest.java index 141ac655457..8984b9e38c2 100644 --- a/processing/src/test/java/org/apache/druid/query/dimension/StringDimensionSelectorForTest.java +++ b/processing/src/test/java/org/apache/druid/query/dimension/StringDimensionSelectorForTest.java @@ -19,9 +19,9 @@ package org.apache.druid.query.dimension; -import com.google.common.base.Predicate; import it.unimi.dsi.fastutil.ints.Int2ObjectMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; @@ -108,9 +108,9 @@ public class StringDimensionSelectorForTest extends AbstractDimensionSelector } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/dimension/TestDimensionSelector.java b/processing/src/test/java/org/apache/druid/query/dimension/TestDimensionSelector.java index e7733974a39..18bc599b33c 100644 --- a/processing/src/test/java/org/apache/druid/query/dimension/TestDimensionSelector.java +++ b/processing/src/test/java/org/apache/druid/query/dimension/TestDimensionSelector.java @@ -19,7 +19,7 @@ package org.apache.druid.query.dimension; -import com.google.common.base.Predicate; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AbstractDimensionSelector; @@ -57,9 +57,9 @@ class TestDimensionSelector extends AbstractDimensionSelector } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override diff --git a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java index 9508de7bcac..39f3e92e0cd 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/InDimFilterTest.java @@ -239,15 +239,15 @@ public class InDimFilterTest extends InitializedNullHandlingTest final ValueMatcher matcher = filter.toFilter().makeMatcher(columnSelectorFactory); // This would throw an exception without InDimFilter's null-checking lambda wrapping. - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); row.put("dim", "foo"); // Now it should match. - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); row.put("dim", "fox"); // Now it *shouldn't* match. - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test diff --git a/processing/src/test/java/org/apache/druid/query/filter/SelectorPredicateFactoryTest.java b/processing/src/test/java/org/apache/druid/query/filter/SelectorPredicateFactoryTest.java index c107d927039..13582d1ae0e 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/SelectorPredicateFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/SelectorPredicateFactoryTest.java @@ -28,7 +28,7 @@ public class SelectorPredicateFactoryTest public void testEqualsContract() { EqualsVerifier.forClass(SelectorPredicateFactory.class) - .withIgnoredFields("initLock", "longPredicate", "floatPredicate", "doublePredicate") + .withIgnoredFields("initLock", "longPredicate", "floatPredicate", "doublePredicate", "isNullUnknown") .usingGetClass() .verify(); } diff --git a/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java b/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java index dce49b129f0..186276e6206 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/vector/VectorValueMatcherColumnProcessorFactoryTest.java @@ -128,6 +128,7 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul EasyMock.expect(selector.getValueCardinality()).andReturn(1024).anyTimes(); EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(false).anyTimes(); EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes(); + EasyMock.expect(selector.lookupName(0)).andReturn(null).anyTimes(); EasyMock.expect(lookup.lookupId("any value")).andReturn(1).anyTimes(); EasyMock.expect(lookup.lookupId("another value")).andReturn(-1).anyTimes(); EasyMock.replay(selector, lookup); @@ -151,9 +152,7 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul Assert.assertEquals(VECTOR_SIZE, matcher.getMaxVectorSize()); Assert.assertEquals(CURRENT_SIZE, matcher.getCurrentVectorSize()); - // value not exist in dictionary uses boolean matcher VectorValueMatcher booleanMatcher = matcherFactory.makeMatcher("another value"); - Assert.assertTrue(booleanMatcher instanceof BooleanVectorValueMatcher); Assert.assertEquals(VECTOR_SIZE, booleanMatcher.getMaxVectorSize()); Assert.assertEquals(CURRENT_SIZE, booleanMatcher.getCurrentVectorSize()); EasyMock.verify(selector, lookup); @@ -165,10 +164,14 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul // cardinality 0 has special path to always use boolean matcher SingleValueDimensionVectorSelector selector = EasyMock.createMock(SingleValueDimensionVectorSelector.class); + IdLookup lookup = EasyMock.createMock(IdLookup.class); EasyMock.expect(selector.getCurrentVectorSize()).andReturn(CURRENT_SIZE).anyTimes(); EasyMock.expect(selector.getMaxVectorSize()).andReturn(VECTOR_SIZE).anyTimes(); EasyMock.expect(selector.getValueCardinality()).andReturn(0).anyTimes(); - EasyMock.replay(selector); + EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes(); + EasyMock.expect(lookup.lookupId("any value")).andReturn(0).anyTimes(); + EasyMock.expect(selector.lookupName(0)).andReturn("any value").anyTimes(); + EasyMock.replay(selector, lookup); VectorValueMatcherFactory matcherFactory = VectorValueMatcherColumnProcessorFactory.instance().makeSingleValueDimensionProcessor( @@ -193,7 +196,7 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul Assert.assertTrue(anotherMatcher instanceof BooleanVectorValueMatcher); Assert.assertEquals(VECTOR_SIZE, anotherMatcher.getMaxVectorSize()); Assert.assertEquals(CURRENT_SIZE, anotherMatcher.getCurrentVectorSize()); - EasyMock.verify(selector); + EasyMock.verify(selector, lookup); } @Test @@ -249,6 +252,7 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul EasyMock.expect(selector.getValueCardinality()).andReturn(1).anyTimes(); EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(false).anyTimes(); EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes(); + EasyMock.expect(selector.lookupName(0)).andReturn(null).anyTimes(); EasyMock.expect(lookup.lookupId("any value")).andReturn(1).anyTimes(); EasyMock.expect(lookup.lookupId(null)).andReturn(0).anyTimes(); EasyMock.replay(selector, lookup); @@ -283,6 +287,7 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul EasyMock.expect(selector.getValueCardinality()).andReturn(11).anyTimes(); EasyMock.expect(selector.nameLookupPossibleInAdvance()).andReturn(false).anyTimes(); EasyMock.expect(selector.idLookup()).andReturn(lookup).anyTimes(); + EasyMock.expect(selector.lookupName(0)).andReturn(null).anyTimes(); EasyMock.expect(lookup.lookupId("any value")).andReturn(-1).anyTimes(); EasyMock.expect(lookup.lookupId(null)).andReturn(0).anyTimes(); EasyMock.replay(selector, lookup); @@ -300,12 +305,10 @@ public class VectorValueMatcherColumnProcessorFactoryTest extends InitializedNul Assert.assertTrue(matcherFactory instanceof MultiValueStringVectorValueMatcher); VectorValueMatcher valueNotExistMatcher = matcherFactory.makeMatcher("any value"); - Assert.assertTrue(valueNotExistMatcher instanceof BooleanVectorValueMatcher); Assert.assertEquals(VECTOR_SIZE, valueNotExistMatcher.getMaxVectorSize()); Assert.assertEquals(CURRENT_SIZE, valueNotExistMatcher.getCurrentVectorSize()); VectorValueMatcher valueExistMatcher = matcherFactory.makeMatcher((String) null); - Assert.assertFalse(valueExistMatcher instanceof BooleanVectorValueMatcher); Assert.assertEquals(VECTOR_SIZE, valueExistMatcher.getMaxVectorSize()); Assert.assertEquals(CURRENT_SIZE, valueExistMatcher.getCurrentVectorSize()); EasyMock.verify(selector, lookup); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java index 6851a2022f8..ee89ae40fa9 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByTimeseriesQueryRunnerTest.java @@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.MapBasedRow; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.HumanReadableBytes; @@ -43,6 +44,8 @@ import org.apache.druid.query.aggregation.ExpressionLambdaAggregatorFactory; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerTest; import org.apache.druid.query.timeseries.TimeseriesResultValue; @@ -364,4 +367,30 @@ public class GroupByTimeseriesQueryRunnerTest extends TimeseriesQueryRunnerTest runner.run(QueryPlus.wrap(query)).toList(); } + + @Override + @Test + public void testTimeseriesWithInvertedFilterOnNonExistentDimension() + { + if (NullHandling.replaceWithDefault()) { + super.testTimeseriesWithInvertedFilterOnNonExistentDimension(); + return; + } + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(QueryRunnerTestHelper.DATA_SOURCE) + .granularity(QueryRunnerTestHelper.DAY_GRAN) + .filters(new NotDimFilter(new SelectorDimFilter("bobby", "sally", null))) + .intervals(QueryRunnerTestHelper.FIRST_TO_THIRD) + .aggregators(aggregatorFactoryList) + .postAggregators(QueryRunnerTestHelper.ADD_ROWS_INDEX_CONSTANT) + .descending(descending) + .context(makeContext()) + .build(); + + + Iterable> results = runner.run(QueryPlus.wrap(query)) + .toList(); + // group by query results are empty instead of day bucket results with zeros and nulls + Assert.assertEquals(Collections.emptyList(), results); + } } diff --git a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java index a89103ac44a..aefad5c6dde 100644 --- a/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/timeseries/TimeseriesQueryRunnerTest.java @@ -140,7 +140,7 @@ public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest .collect(Collectors.toList()); } - private void assertExpectedResults(Iterable> expectedResults, Iterable> results) + protected void assertExpectedResults(Iterable> expectedResults, Iterable> results) { if (descending) { expectedResults = TestHelper.revert(expectedResults); @@ -151,7 +151,7 @@ public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest protected final QueryRunner> runner; protected final boolean descending; protected final boolean vectorize; - private final List aggregatorFactoryList; + protected final List aggregatorFactoryList; public TimeseriesQueryRunnerTest( QueryRunner> runner, @@ -1767,30 +1767,49 @@ public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest .context(makeContext()) .build(); - List> expectedResults = Arrays.asList( - new Result<>( - DateTimes.of("2011-04-01"), - new TimeseriesResultValue( - ImmutableMap.of( - "rows", 13L, - "index", 6626.151596069336, - "addRowsIndexConstant", 6640.151596069336, - "uniques", QueryRunnerTestHelper.UNIQUES_9 - ) - ) - ), - new Result<>( - DateTimes.of("2011-04-02"), - new TimeseriesResultValue( - ImmutableMap.of( - "rows", 13L, - "index", 5833.2095947265625, - "addRowsIndexConstant", 5847.2095947265625, - "uniques", QueryRunnerTestHelper.UNIQUES_9 - ) - ) - ) - ); + + List> expectedResults; + if (NullHandling.sqlCompatible()) { + expectedResults = Arrays.asList( + new Result<>( + DateTimes.of("2011-04-01"), + new TimeseriesResultValue( + TestHelper.makeMap("rows", 0L, "index", null, "uniques", 0.0, "addRowsIndexConstant", null) + ) + ), + new Result<>( + DateTimes.of("2011-04-02"), + new TimeseriesResultValue( + TestHelper.makeMap("rows", 0L, "index", null, "uniques", 0.0, "addRowsIndexConstant", null) + ) + ) + ); + } else { + expectedResults = Arrays.asList( + new Result<>( + DateTimes.of("2011-04-01"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", 13L, + "index", 6626.151596069336, + "addRowsIndexConstant", 6640.151596069336, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ) + ) + ), + new Result<>( + DateTimes.of("2011-04-02"), + new TimeseriesResultValue( + ImmutableMap.of( + "rows", 13L, + "index", 5833.2095947265625, + "addRowsIndexConstant", 5847.2095947265625, + "uniques", QueryRunnerTestHelper.UNIQUES_9 + ) + ) + ) + ); + } Iterable> results = runner.run(QueryPlus.wrap(query)) .toList(); diff --git a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java index 0174b845250..f68104ff481 100644 --- a/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java +++ b/processing/src/test/java/org/apache/druid/query/topn/TopNMetricSpecOptimizationsTest.java @@ -19,7 +19,6 @@ package org.apache.druid.query.topn; -import com.google.common.base.Predicate; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.java.util.common.DateTimes; @@ -32,6 +31,7 @@ import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.DoubleMaxAggregatorFactory; import org.apache.druid.query.aggregation.DoubleMinAggregatorFactory; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; @@ -362,7 +362,7 @@ public class TopNMetricSpecOptimizationsTest } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { return null; } diff --git a/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java index e042fd39b74..4065a15a047 100644 --- a/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ColumnSelectorColumnIndexSelectorTest.java @@ -82,7 +82,7 @@ public class ColumnSelectorColumnIndexSelectorTest EasyMock.expect(valueIndex.getBitmap(0)).andReturn(someBitmap).anyTimes(); EasyMock.expect(someIndex.forValue("foo")).andReturn(columnIndex).anyTimes(); - EasyMock.expect(columnIndex.computeBitmapResult(EasyMock.anyObject())).andReturn(someBitmap).anyTimes(); + EasyMock.expect(columnIndex.computeBitmapResult(EasyMock.anyObject(), EasyMock.eq(false))).andReturn(someBitmap).anyTimes(); ColumnHolder nonStringHolder = EasyMock.createMock(ColumnHolder.class); EasyMock.expect(index.getColumnHolder(NON_STRING_DICTIONARY_COLUMN_NAME)).andReturn(nonStringHolder).anyTimes(); @@ -112,7 +112,8 @@ public class ColumnSelectorColumnIndexSelectorTest Assert.assertNotNull(valueIndex); ImmutableBitmap valueBitmap = valueIndex.forValue("foo") .computeBitmapResult( - new DefaultBitmapResultFactory(indexSelector.getBitmapFactory()) + new DefaultBitmapResultFactory(indexSelector.getBitmapFactory()), + false ); Assert.assertNotNull(valueBitmap); EasyMock.verify(bitmapFactory, virtualColumns, index, indexSupplier); diff --git a/processing/src/test/java/org/apache/druid/segment/ConstantDimensionSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/ConstantDimensionSelectorTest.java index de305f0a076..0391c6cf345 100644 --- a/processing/src/test/java/org/apache/druid/segment/ConstantDimensionSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ConstantDimensionSelectorTest.java @@ -22,11 +22,17 @@ package org.apache.druid.segment; import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.extraction.StringFormatExtractionFn; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; +import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.segment.data.IndexedInts; +import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; import org.junit.Test; -public class ConstantDimensionSelectorTest +import java.util.Objects; + +public class ConstantDimensionSelectorTest extends InitializedNullHandlingTest { private final DimensionSelector NULL_SELECTOR = DimensionSelector.constant(null); private final DimensionSelector CONST_SELECTOR = DimensionSelector.constant("billy"); @@ -88,4 +94,18 @@ public class ConstantDimensionSelectorTest Assert.assertEquals(0, CONST_EXTRACTION_SELECTOR.idLookup().lookupId("billy")); Assert.assertEquals(-1, CONST_EXTRACTION_SELECTOR.idLookup().lookupId("bob")); } + + @Test + public void testValueMatcherPredicates() + { + DruidPredicateFactory nullUnkown = new StringPredicateDruidPredicateFactory(Objects::nonNull, true); + ValueMatcher matcher = NULL_SELECTOR.makeValueMatcher(nullUnkown); + Assert.assertFalse(matcher.matches(false)); + Assert.assertTrue(matcher.matches(true)); + + DruidPredicateFactory notUnknown = new StringPredicateDruidPredicateFactory(Objects::nonNull, false); + matcher = NULL_SELECTOR.makeValueMatcher(notUnknown); + Assert.assertFalse(matcher.matches(false)); + Assert.assertFalse(matcher.matches(true)); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/ConstantMultiValueDimensionSelectorTest.java b/processing/src/test/java/org/apache/druid/segment/ConstantMultiValueDimensionSelectorTest.java index 47d2b9966e7..f8be9019c0e 100644 --- a/processing/src/test/java/org/apache/druid/segment/ConstantMultiValueDimensionSelectorTest.java +++ b/processing/src/test/java/org/apache/druid/segment/ConstantMultiValueDimensionSelectorTest.java @@ -23,6 +23,7 @@ import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import org.apache.druid.query.extraction.StringFormatExtractionFn; import org.apache.druid.query.extraction.SubstringDimExtractionFn; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.segment.data.IndexedInts; import org.apache.druid.testing.InitializedNullHandlingTest; import org.junit.Assert; @@ -120,38 +121,38 @@ public class ConstantMultiValueDimensionSelectorTest extends InitializedNullHand @Test public void testValueMatcher() { - Assert.assertTrue(NULL_SELECTOR.makeValueMatcher((String) null).matches()); - Assert.assertFalse(NULL_SELECTOR.makeValueMatcher("douglas").matches()); + Assert.assertTrue(NULL_SELECTOR.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(NULL_SELECTOR.makeValueMatcher("douglas").matches(false)); - Assert.assertTrue(EMPTY_SELECTOR.makeValueMatcher((String) null).matches()); - Assert.assertFalse(EMPTY_SELECTOR.makeValueMatcher("douglas").matches()); + Assert.assertTrue(EMPTY_SELECTOR.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(EMPTY_SELECTOR.makeValueMatcher("douglas").matches(false)); - Assert.assertTrue(CONST_SELECTOR.makeValueMatcher("billy").matches()); - Assert.assertTrue(CONST_SELECTOR.makeValueMatcher("douglas").matches()); - Assert.assertFalse(CONST_SELECTOR.makeValueMatcher("debbie").matches()); + Assert.assertTrue(CONST_SELECTOR.makeValueMatcher("billy").matches(false)); + Assert.assertTrue(CONST_SELECTOR.makeValueMatcher("douglas").matches(false)); + Assert.assertFalse(CONST_SELECTOR.makeValueMatcher("debbie").matches(false)); - Assert.assertTrue(NULL_EXTRACTION_SELECTOR.makeValueMatcher("billy").matches()); - Assert.assertFalse(NULL_EXTRACTION_SELECTOR.makeValueMatcher((String) null).matches()); + Assert.assertTrue(NULL_EXTRACTION_SELECTOR.makeValueMatcher("billy").matches(false)); + Assert.assertFalse(NULL_EXTRACTION_SELECTOR.makeValueMatcher((String) null).matches(false)); - Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher("bill").matches()); - Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher("doug").matches()); - Assert.assertFalse(CONST_EXTRACTION_SELECTOR.makeValueMatcher("billy").matches()); + Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher("bill").matches(false)); + Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher("doug").matches(false)); + Assert.assertFalse(CONST_EXTRACTION_SELECTOR.makeValueMatcher("billy").matches(false)); - Assert.assertTrue(NULL_SELECTOR.makeValueMatcher(Predicates.isNull()).matches()); - Assert.assertFalse(NULL_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches()); + Assert.assertTrue(NULL_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.isNull(), false)).matches(false)); + Assert.assertFalse(NULL_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("billy")).matches(false)); - Assert.assertTrue(EMPTY_SELECTOR.makeValueMatcher(Predicates.equalTo(null)).matches()); - Assert.assertFalse(EMPTY_SELECTOR.makeValueMatcher(Predicates.equalTo("douglas")).matches()); + Assert.assertTrue(EMPTY_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(null)).matches(false)); + Assert.assertFalse(EMPTY_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("douglas")).matches(false)); - Assert.assertTrue(CONST_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches()); - Assert.assertTrue(CONST_SELECTOR.makeValueMatcher(Predicates.equalTo("douglas")).matches()); - Assert.assertFalse(CONST_SELECTOR.makeValueMatcher(Predicates.equalTo("debbie")).matches()); + Assert.assertTrue(CONST_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("billy")).matches(false)); + Assert.assertTrue(CONST_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("douglas")).matches(false)); + Assert.assertFalse(CONST_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("debbie")).matches(false)); - Assert.assertTrue(NULL_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches()); - Assert.assertFalse(NULL_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo(null)).matches()); + Assert.assertTrue(NULL_EXTRACTION_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("billy")).matches(false)); + Assert.assertFalse(NULL_EXTRACTION_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(null)).matches(false)); - Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("bill")).matches()); - Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("doug")).matches()); - Assert.assertFalse(CONST_EXTRACTION_SELECTOR.makeValueMatcher(Predicates.equalTo("billy")).matches()); + Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("bill")).matches(false)); + Assert.assertTrue(CONST_EXTRACTION_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("doug")).matches(false)); + Assert.assertFalse(CONST_EXTRACTION_SELECTOR.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo("billy")).matches(false)); } } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 664a28603cb..5990ba6513c 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -358,7 +358,7 @@ public class IndexBuilder ), AggregatorFactory.class ), - null, + schema.getDimensionsSpec(), new File(tmpDir, StringUtils.format("testIndex-%s", UUID.randomUUID())), indexSpec, indexSpec, diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java index 3b3706c6ea5..9e2a69ff364 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerNullHandlingTest.java @@ -205,7 +205,8 @@ public class IndexMergerNullHandlingTest new DefaultBitmapResultFactory( indexSpec.getBitmapSerdeFactory() .getBitmapFactory() - ) + ), + false ); final List actualNullRows = new ArrayList<>(); final IntIterator iterator = nullBitmap.iterator(); diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java index 5cc10619a72..7f1e227f1ec 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerTestBase.java @@ -133,7 +133,7 @@ public class IndexMergerTestBase extends InitializedNullHandlingTest } return new ImmutableBitmapValues(index.forValue(value).computeBitmapResult( - new DefaultBitmapResultFactory(adapter.getQueryableIndex().getBitmapFactoryForDimensions())) + new DefaultBitmapResultFactory(adapter.getQueryableIndex().getBitmapFactoryForDimensions()), false) ); } diff --git a/processing/src/test/java/org/apache/druid/segment/ListCursor.java b/processing/src/test/java/org/apache/druid/segment/ListCursor.java index 2ff5d78dab2..fc1eba743d4 100644 --- a/processing/src/test/java/org/apache/druid/segment/ListCursor.java +++ b/processing/src/test/java/org/apache/druid/segment/ListCursor.java @@ -19,9 +19,9 @@ package org.apache.druid.segment; -import com.google.common.base.Predicate; import org.apache.druid.query.BaseQuery; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnCapabilities; @@ -72,7 +72,7 @@ public class ListCursor implements Cursor } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { throw new UnsupportedOperationException(); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java index 59822b03d55..723bdd07f06 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/BaseFilterTest.java @@ -62,6 +62,7 @@ import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.AutoTypeColumnSchema; @@ -136,6 +137,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest new ExpressionVirtualColumn("exprLong", "1 + 2", ColumnType.LONG, TestExprMacroTable.INSTANCE), new ExpressionVirtualColumn("vdim0", "dim0", ColumnType.STRING, TestExprMacroTable.INSTANCE), new ExpressionVirtualColumn("vdim1", "dim1", ColumnType.STRING, TestExprMacroTable.INSTANCE), + new ExpressionVirtualColumn("vs0", "s0", ColumnType.STRING, TestExprMacroTable.INSTANCE), new ExpressionVirtualColumn("vd0", "d0", ColumnType.DOUBLE, TestExprMacroTable.INSTANCE), new ExpressionVirtualColumn("vf0", "f0", ColumnType.FLOAT, TestExprMacroTable.INSTANCE), new ExpressionVirtualColumn("vl0", "l0", ColumnType.LONG, TestExprMacroTable.INSTANCE), @@ -149,13 +151,18 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest static final TimestampSpec DEFAULT_TIMESTAMP_SPEC = new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")); static final DimensionsSpec DEFAULT_DIM_SPEC = new DimensionsSpec( ImmutableList.builder() - .addAll(DimensionsSpec.getDefaultSchemas(ImmutableList.of( - "dim0", - "dim1", - "dim2", - "dim3", - "timeDim" - ))) + .addAll( + DimensionsSpec.getDefaultSchemas( + ImmutableList.of( + "dim0", + "dim1", + "dim2", + "dim3", + "timeDim", + "s0" + ) + ) + ) .add(new DoubleDimensionSchema("d0")) .add(new FloatDimensionSchema("f0")) .add(new LongDimensionSchema("l0")) @@ -180,6 +187,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest .add("dim1", ColumnType.STRING) .add("dim2", ColumnType.STRING) .add("timeDim", ColumnType.STRING) + .add("s0", ColumnType.STRING) .add("d0", ColumnType.DOUBLE) .add("f0", ColumnType.FLOAT) .add("l0", ColumnType.LONG) @@ -195,6 +203,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest "", ImmutableList.of("a", "b"), "2017-07-25", + "", 0.0, 0.0f, 0L, @@ -208,6 +217,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest "10", ImmutableList.of(), "2017-07-25", + "a", 10.1, 10.1f, 100L, @@ -221,6 +231,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest "2", ImmutableList.of(""), "2017-05-25", + "b", null, 5.5f, 40L, @@ -234,6 +245,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest "1", ImmutableList.of("a"), "2020-01-25", + null, 120.0245, 110.0f, null, @@ -247,6 +259,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest "abdef", ImmutableList.of("c"), null, + "c", 60.0, null, 9001L, @@ -260,6 +273,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest "abc", null, "2020-01-25", + "a", 765.432, 123.45f, 12345L, @@ -283,7 +297,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest } - static InputRow makeSchemaRow( + public static InputRow makeSchemaRow( final InputRowParser> parser, final RowSignature signature, @Nullable Object... elements @@ -1034,7 +1048,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest final List values = new ArrayList<>(); for (InputRow row : rows) { rowSupplier.set(row); - if (matcher.matches()) { + if (matcher.matches(false)) { values.add((String) row.getRaw(selectColumn)); } } @@ -1055,6 +1069,10 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest && !(adapter instanceof FrameStorageAdapter); assertFilterMatches(filter, expectedRows, testVectorized); + // test double inverted + if (!StringUtils.toLowerCase(testName).contains("concise")) { + assertFilterMatches(NotDimFilter.of(NotDimFilter.of(filter)), expectedRows, testVectorized); + } } protected void assertFilterMatchesSkipArrays( @@ -1078,6 +1096,10 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest Assert.assertTrue(t.getMessage().contains("ARRAY")); } else { assertFilterMatches(filter, expectedRows, testVectorized); + // test double inverted + if (!StringUtils.toLowerCase(testName).contains("concise")) { + assertFilterMatches(NotDimFilter.of(NotDimFilter.of(filter)), expectedRows, testVectorized); + } } } @@ -1087,6 +1109,10 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest ) { assertFilterMatches(filter, expectedRows, false); + // test double inverted + if (!StringUtils.toLowerCase(testName).contains("concise")) { + assertFilterMatches(NotDimFilter.of(NotDimFilter.of(filter)), expectedRows, false); + } } private void assertFilterMatches( 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 933843abb25..47523576e0b 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 @@ -51,8 +51,8 @@ public class BoundFilterTest extends BaseFilterTest { private static final List ROWS = ImmutableList.builder() .addAll(DEFAULT_ROWS) - .add(makeDefaultSchemaRow("6", "-1000", ImmutableList.of("a"), null, 6.6, null, 10L)) - .add(makeDefaultSchemaRow("7", "-10.012", ImmutableList.of("d"), null, null, 3.0f, null)) + .add(makeDefaultSchemaRow("6", "-1000", ImmutableList.of("a"), null, null, 6.6, null, 10L)) + .add(makeDefaultSchemaRow("7", "-10.012", ImmutableList.of("d"), null, "e", null, 3.0f, null)) .build(); public BoundFilterTest( @@ -892,7 +892,12 @@ public class BoundFilterTest extends BaseFilterTest { EqualsVerifier.forClass(BoundFilter.BoundDimFilterDruidPredicateFactory.class) .usingGetClass() - .withIgnoredFields("longPredicateSupplier", "floatPredicateSupplier", "doublePredicateSupplier") + .withIgnoredFields( + "longPredicateSupplier", + "floatPredicateSupplier", + "doublePredicateSupplier", + "isNullUnknown" + ) .verify(); } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/DimensionPredicateFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/DimensionPredicateFilterTest.java index 07707a5d887..5db99a19054 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/DimensionPredicateFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/DimensionPredicateFilterTest.java @@ -37,7 +37,7 @@ public class DimensionPredicateFilterTest public void testEqualsContractForDelegatingStringPredicateFactory() { EqualsVerifier.forClass(DimensionPredicateFilter.DelegatingStringPredicateFactory.class) - .withIgnoredFields("baseStringPredicate") + .withIgnoredFields("baseStringPredicate", "isNullUnknown") .usingGetClass() .verify(); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java index 8f9cc38aa09..5403ac7aca6 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/EqualityFilterTests.java @@ -37,6 +37,7 @@ import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.EqualityFilter; import org.apache.druid.query.filter.FilterTuning; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; @@ -80,12 +81,32 @@ public class EqualityFilterTests { if (NullHandling.sqlCompatible()) { assertFilterMatches(new EqualityFilter("dim0", ColumnType.STRING, "", null), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim0", ColumnType.STRING, "", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); } assertFilterMatches(new EqualityFilter("dim0", ColumnType.STRING, "0", null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim0", ColumnType.STRING, "0", null)), + ImmutableList.of("1", "2", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("dim0", ColumnType.STRING, "1", null), ImmutableList.of("1")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim0", ColumnType.STRING, "1", null)), + ImmutableList.of("0", "2", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("dim0", ColumnType.LONG, 0L, null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim0", ColumnType.LONG, 0L, null)), + ImmutableList.of("1", "2", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("dim0", ColumnType.LONG, 1L, null), ImmutableList.of("1")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim0", ColumnType.LONG, 1L, null)), + ImmutableList.of("0", "2", "3", "4", "5") + ); } @Test @@ -107,18 +128,42 @@ public class EqualityFilterTests new EqualityFilter("allow-dim0", ColumnType.STRING, "1", null), ImmutableList.of() ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("allow-dim0", ColumnType.STRING, "1", null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("3", "4") + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); assertFilterMatchesSkipVectorize( new EqualityFilter("allow-dim0", ColumnType.STRING, "4", null), ImmutableList.of("4") ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("allow-dim0", ColumnType.STRING, "4", null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("3") + : ImmutableList.of("0", "1", "2", "3", "5") + ); assertFilterMatchesSkipVectorize( new EqualityFilter("deny-dim0", ColumnType.STRING, "0", null), ImmutableList.of("0") ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("deny-dim0", ColumnType.STRING, "0", null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "2", "5") + : ImmutableList.of("1", "2", "3", "4", "5") + ); assertFilterMatchesSkipVectorize( new EqualityFilter("deny-dim0", ColumnType.STRING, "4", null), ImmutableList.of() ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("deny-dim0", ColumnType.STRING, "4", null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "2", "5") + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); // auto ingests arrays instead of MVDs which dont work with list filtered virtual column if (!isAutoSchema()) { @@ -126,18 +171,41 @@ public class EqualityFilterTests new EqualityFilter("allow-dim2", ColumnType.STRING, "b", null), ImmutableList.of() ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("allow-dim2", ColumnType.STRING, "b", null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "3") + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); assertFilterMatchesSkipVectorize( new EqualityFilter("allow-dim2", ColumnType.STRING, "a", null), ImmutableList.of("0", "3") ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("allow-dim2", ColumnType.STRING, "a", null)), + NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("1", "2", "4", "5") + ); assertFilterMatchesSkipVectorize( new EqualityFilter("deny-dim2", ColumnType.STRING, "b", null), ImmutableList.of("0") ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("deny-dim2", ColumnType.STRING, "b", null)), + NullHandling.replaceWithDefault() + ? ImmutableList.of("1", "2", "3", "4", "5") + : ImmutableList.of("2", "4") + ); assertFilterMatchesSkipVectorize( new EqualityFilter("deny-dim2", ColumnType.STRING, "a", null), ImmutableList.of() ); + // mvds are strange + assertFilterMatchesSkipVectorize( + NotDimFilter.of(new EqualityFilter("deny-dim2", ColumnType.STRING, "a", null)), + NullHandling.replaceWithDefault() + ? ImmutableList.of("0", "1", "2", "3", "4", "5") + : ImmutableList.of("0", "2", "4") + ); } } @@ -145,14 +213,82 @@ public class EqualityFilterTests public void testSingleValueStringColumnWithNulls() { if (NullHandling.sqlCompatible()) { - assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "", null), ImmutableList.of("0")); + assertFilterMatches( + new EqualityFilter("dim1", ColumnType.STRING, "", null), + ImmutableList.of("0") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim1", ColumnType.STRING, "", null)), + ImmutableList.of("1", "2", "3", "4", "5") + ); + assertFilterMatches( + new EqualityFilter("s0", ColumnType.STRING, "", null), + ImmutableList.of("0") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "", null)), + ImmutableList.of("1", "2", "4", "5") + ); } assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "10", null), ImmutableList.of("1")); assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "2", null), ImmutableList.of("2")); assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "1", null), ImmutableList.of("3")); assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "abdef", null), ImmutableList.of("4")); assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "abc", null), ImmutableList.of("5")); - assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "ab", null), ImmutableList.of()); + assertFilterMatches(new EqualityFilter("dim1", ColumnType.STRING, "noexist", null), ImmutableList.of()); + + assertFilterMatches(new EqualityFilter("s0", ColumnType.STRING, "a", null), ImmutableList.of("1", "5")); + assertFilterMatches(new EqualityFilter("s0", ColumnType.STRING, "b", null), ImmutableList.of("2")); + assertFilterMatches(new EqualityFilter("s0", ColumnType.STRING, "c", null), ImmutableList.of("4")); + assertFilterMatches(new EqualityFilter("s0", ColumnType.STRING, "noexist", null), ImmutableList.of()); + + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim1", ColumnType.STRING, "10", null)), + ImmutableList.of("0", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim1", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), + ImmutableList.of("0", "2", "4") + ); + + try { + // make sure if 3vl is disabled with behave with 2vl + NullHandling.initializeForTestsWithValues(false, false, null); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), + ImmutableList.of("0", "2", "3", "4") + ); + } + finally { + NullHandling.initializeForTests(); + } + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "4", "5") + ); + } else { + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim1", ColumnType.STRING, "10", null)), + ImmutableList.of("0", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim1", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "a", null)), + ImmutableList.of("0", "2", "3", "4") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("s0", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + } } @Test @@ -160,14 +296,70 @@ public class EqualityFilterTests { // testSingleValueStringColumnWithNulls but with virtual column selector if (NullHandling.sqlCompatible()) { - assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "", null), ImmutableList.of("0")); + assertFilterMatches( + new EqualityFilter("vdim1", ColumnType.STRING, "", null), + ImmutableList.of("0") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vdim1", ColumnType.STRING, "", null)), + ImmutableList.of("1", "2", "3", "4", "5") + ); + assertFilterMatches( + new EqualityFilter("vs0", ColumnType.STRING, "", null), + ImmutableList.of("0") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vs0", ColumnType.STRING, "", null)), + ImmutableList.of("1", "2", "4", "5") + ); } assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "10", null), ImmutableList.of("1")); assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "2", null), ImmutableList.of("2")); assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "1", null), ImmutableList.of("3")); assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "abdef", null), ImmutableList.of("4")); assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "abc", null), ImmutableList.of("5")); - assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "ab", null), ImmutableList.of()); + assertFilterMatches(new EqualityFilter("vdim1", ColumnType.STRING, "noexist", null), ImmutableList.of()); + + assertFilterMatches(new EqualityFilter("vs0", ColumnType.STRING, "a", null), ImmutableList.of("1", "5")); + assertFilterMatches(new EqualityFilter("vs0", ColumnType.STRING, "b", null), ImmutableList.of("2")); + assertFilterMatches(new EqualityFilter("vs0", ColumnType.STRING, "c", null), ImmutableList.of("4")); + assertFilterMatches(new EqualityFilter("vs0", ColumnType.STRING, "noexist", null), ImmutableList.of()); + + if (NullHandling.sqlCompatible()) { + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vdim1", ColumnType.STRING, "10", null)), + ImmutableList.of("0", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vdim1", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vs0", ColumnType.STRING, "a", null)), + ImmutableList.of("0", "2", "4") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vs0", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "4", "5") + ); + } else { + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vdim1", ColumnType.STRING, "10", null)), + ImmutableList.of("0", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vdim1", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vs0", ColumnType.STRING, "a", null)), + ImmutableList.of("0", "2", "3", "4") + ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vs0", ColumnType.STRING, "noexist", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + } } @Test @@ -218,14 +410,26 @@ public class EqualityFilterTests } else { if (NullHandling.sqlCompatible()) { assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "", null), ImmutableList.of("2")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim2", ColumnType.STRING, "", null)), + ImmutableList.of("0", "3", "4") + ); } assertFilterMatches( new EqualityFilter("dim2", ColumnType.STRING, "a", null), ImmutableList.of("0", "3") ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim2", ColumnType.STRING, "a", null)), + NullHandling.replaceWithDefault() ? ImmutableList.of("1", "2", "4", "5") : ImmutableList.of("2", "4") + ); assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "b", null), ImmutableList.of("0")); assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "c", null), ImmutableList.of("4")); assertFilterMatches(new EqualityFilter("dim2", ColumnType.STRING, "d", null), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim2", ColumnType.STRING, "d", null)), + NullHandling.replaceWithDefault() ? ImmutableList.of("0", "1", "2", "3", "4", "5") : ImmutableList.of("0", "2", "3", "4") + ); } } @@ -238,6 +442,13 @@ public class EqualityFilterTests assertFilterMatches(new EqualityFilter("dim3", ColumnType.STRING, "a", null), ImmutableList.of()); assertFilterMatches(new EqualityFilter("dim3", ColumnType.STRING, "b", null), ImmutableList.of()); assertFilterMatches(new EqualityFilter("dim3", ColumnType.STRING, "c", null), ImmutableList.of()); + + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim3", ColumnType.STRING, "c", null)), + NullHandling.sqlCompatible() + ? ImmutableList.of() + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); } @Test @@ -249,6 +460,12 @@ public class EqualityFilterTests assertFilterMatches(new EqualityFilter("dim4", ColumnType.STRING, "a", null), ImmutableList.of()); assertFilterMatches(new EqualityFilter("dim4", ColumnType.STRING, "b", null), ImmutableList.of()); assertFilterMatches(new EqualityFilter("dim4", ColumnType.STRING, "c", null), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("dim4", ColumnType.STRING, "c", null)), + NullHandling.sqlCompatible() + ? ImmutableList.of() + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); } @Test @@ -258,6 +475,10 @@ public class EqualityFilterTests new EqualityFilter("expr", ColumnType.STRING, "1.1", null), ImmutableList.of("0", "1", "2", "3", "4", "5") ); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("expr", ColumnType.STRING, "1.1", null)), + ImmutableList.of() + ); assertFilterMatches(new EqualityFilter("expr", ColumnType.STRING, "1.2", null), ImmutableList.of()); assertFilterMatches( @@ -278,6 +499,10 @@ public class EqualityFilterTests ImmutableList.of("0", "1", "2", "3", "4", "5") ); assertFilterMatches(new EqualityFilter("expr", ColumnType.DOUBLE, "1.2", null), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("expr", ColumnType.DOUBLE, "1.2", null)), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); } @Test @@ -285,15 +510,47 @@ public class EqualityFilterTests { if (canTestNumericNullsAsDefaultValues) { assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 0f, null), ImmutableList.of("0", "4")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("f0", ColumnType.FLOAT, 0f, null)), + ImmutableList.of("1", "2", "3", "5") + ); assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 0.0, null), ImmutableList.of("0", "2")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("d0", ColumnType.DOUBLE, 0.0, null)), + ImmutableList.of("1", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 0L, null), ImmutableList.of("0", "3")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("l0", ColumnType.LONG, 0L, null)), + ImmutableList.of("1", "2", "4", "5") + ); + assertFilterMatches(new EqualityFilter("f0", ColumnType.STRING, "0", null), ImmutableList.of("0", "4")); assertFilterMatches(new EqualityFilter("d0", ColumnType.STRING, "0", null), ImmutableList.of("0", "2")); assertFilterMatches(new EqualityFilter("l0", ColumnType.STRING, "0", null), ImmutableList.of("0", "3")); } else { assertFilterMatches(new EqualityFilter("f0", ColumnType.FLOAT, 0f, null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("f0", ColumnType.FLOAT, 0f, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "2", "3", "5") + : ImmutableList.of("1", "2", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("d0", ColumnType.DOUBLE, 0.0, null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("d0", ColumnType.DOUBLE, 0.0, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "3", "4", "5") + : ImmutableList.of("1", "2", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("l0", ColumnType.LONG, 0L, null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("l0", ColumnType.LONG, 0L, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "2", "4", "5") + : ImmutableList.of("1", "2", "3", "4", "5") + ); + assertFilterMatches(new EqualityFilter("f0", ColumnType.STRING, "0", null), ImmutableList.of("0")); assertFilterMatches(new EqualityFilter("d0", ColumnType.STRING, "0", null), ImmutableList.of("0")); assertFilterMatches(new EqualityFilter("l0", ColumnType.STRING, "0", null), ImmutableList.of("0")); @@ -305,15 +562,47 @@ public class EqualityFilterTests { if (canTestNumericNullsAsDefaultValues) { assertFilterMatches(new EqualityFilter("vf0", ColumnType.FLOAT, 0f, null), ImmutableList.of("0", "4")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vf0", ColumnType.FLOAT, 0f, null)), + ImmutableList.of("1", "2", "3", "5") + ); assertFilterMatches(new EqualityFilter("vd0", ColumnType.DOUBLE, 0.0, null), ImmutableList.of("0", "2")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vd0", ColumnType.DOUBLE, 0.0, null)), + ImmutableList.of("1", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("vl0", ColumnType.LONG, 0L, null), ImmutableList.of("0", "3")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vl0", ColumnType.LONG, 0L, null)), + ImmutableList.of("1", "2", "4", "5") + ); + assertFilterMatches(new EqualityFilter("vf0", ColumnType.STRING, "0", null), ImmutableList.of("0", "4")); assertFilterMatches(new EqualityFilter("vd0", ColumnType.STRING, "0", null), ImmutableList.of("0", "2")); assertFilterMatches(new EqualityFilter("vl0", ColumnType.STRING, "0", null), ImmutableList.of("0", "3")); } else { assertFilterMatches(new EqualityFilter("vf0", ColumnType.FLOAT, 0f, null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vf0", ColumnType.FLOAT, 0f, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "2", "3", "5") + : ImmutableList.of("1", "2", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("vd0", ColumnType.DOUBLE, 0.0, null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vd0", ColumnType.DOUBLE, 0.0, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "3", "4", "5") + : ImmutableList.of("1", "2", "3", "4", "5") + ); assertFilterMatches(new EqualityFilter("vl0", ColumnType.LONG, 0L, null), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(new EqualityFilter("vl0", ColumnType.LONG, 0L, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "2", "4", "5") + : ImmutableList.of("1", "2", "3", "4", "5") + ); + assertFilterMatches(new EqualityFilter("vf0", ColumnType.STRING, "0", null), ImmutableList.of("0")); assertFilterMatches(new EqualityFilter("vd0", ColumnType.STRING, "0", null), ImmutableList.of("0")); assertFilterMatches(new EqualityFilter("vl0", ColumnType.STRING, "0", null), ImmutableList.of("0")); @@ -410,6 +699,19 @@ public class EqualityFilterTests ), ImmutableList.of("0", "3") ); + assertFilterMatches( + NotDimFilter.of( + new EqualityFilter( + "arrayString", + ColumnType.STRING_ARRAY, + ImmutableList.of("a", "b", "c"), + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "4", "5") + : ImmutableList.of("1", "2", "4", "5") + ); assertFilterMatches( new EqualityFilter( "arrayString", @@ -446,6 +748,19 @@ public class EqualityFilterTests ), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of( + new EqualityFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{null, null}, + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "3", "4", "5") + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); assertFilterMatches( @@ -457,6 +772,19 @@ public class EqualityFilterTests ), ImmutableList.of("0", "2") ); + assertFilterMatches( + NotDimFilter.of( + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + ImmutableList.of(1L, 2L, 3L), + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "4", "5") + : ImmutableList.of("1", "3", "4", "5") + ); assertFilterMatches( new EqualityFilter( "arrayLong", @@ -493,6 +821,19 @@ public class EqualityFilterTests ), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of( + new EqualityFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + new Object[]{null, null}, + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "2", "4", "5") + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); // test loss of precision matching long arrays with double array match values assertFilterMatches( @@ -533,6 +874,19 @@ public class EqualityFilterTests ), ImmutableList.of("0", "1") ); + assertFilterMatches( + NotDimFilter.of( + new EqualityFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + ImmutableList.of(1.1, 2.2, 3.3), + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("2", "3", "4") + : ImmutableList.of("2", "3", "4", "5") + ); assertFilterMatches( new EqualityFilter( "arrayDouble", @@ -569,6 +923,19 @@ public class EqualityFilterTests ), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of( + new EqualityFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + ImmutableList.of(1.1, 2.2, 3.4), + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "2", "3", "4") + : ImmutableList.of("0", "1", "2", "3", "4", "5") + ); } } @@ -595,6 +962,17 @@ public class EqualityFilterTests ), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of( + new EqualityFilter( + "variant", + ColumnType.STRING_ARRAY, + ImmutableList.of("a", "b", "c"), + null + ) + ), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); assertFilterMatches( new EqualityFilter( @@ -637,7 +1015,6 @@ public class EqualityFilterTests ); } } - } public static class EqualityFilterNonParameterizedTests extends InitializedNullHandlingTest diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java index 3716f6c5f29..e9fea68c0df 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterNonStrictBooleansTest.java @@ -20,11 +20,15 @@ package org.apache.druid.segment.filter; import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.Pair; import org.apache.druid.math.expr.ExpressionProcessing; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.StorageAdapter; import org.junit.Before; +import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -50,4 +54,90 @@ public class ExpressionFilterNonStrictBooleansTest extends ExpressionFilterTest { ExpressionProcessing.initializeForStrictBooleansTests(false); } + + @Override + @Test + public void testComplement() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(edf("dim5 == 'a'"), ImmutableList.of("0")); + // non-strict mode is wild + assertFilterMatches( + NotDimFilter.of(edf("dim5 == 'a'")), + ImmutableList.of("1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + assertFilterMatches( + edf("dim5 == ''"), ImmutableList.of("4") + ); + // non-strict mode! + assertFilterMatches( + NotDimFilter.of(edf("dim5 == ''")), ImmutableList.of("0", "1", "2", "3", "5", "6", "7", "8", "9") + ); + } else { + assertFilterMatches(edf("dim5 == 'a'"), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(edf("dim5 == 'a'")), + ImmutableList.of("1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + } + } + + @Override + @Test + public void testMissingColumn() + { + if (NullHandling.replaceWithDefault()) { + assertFilterMatches( + edf("missing == ''"), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + assertFilterMatches( + edf("missing == otherMissing"), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + } else { + // AS per SQL standard null == null returns false. + assertFilterMatches(edf("missing == null"), ImmutableList.of()); + // in non-strict mode, madness happens + assertFilterMatches( + NotDimFilter.of(edf("missing == null")), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + // also this madness doesn't do madness + assertFilterMatches( + edf("missing == otherMissing"), + ImmutableList.of() + ); + assertFilterMatches( + NotDimFilter.of(edf("missing == otherMissing")), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + } + assertFilterMatches(edf("missing == '1'"), ImmutableList.of()); + assertFilterMatches(edf("missing == 2"), ImmutableList.of()); + if (NullHandling.replaceWithDefault()) { + // missing equivaluent to 0 + assertFilterMatches( + edf("missing < '2'"), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + assertFilterMatches( + edf("missing < 2"), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + assertFilterMatches( + edf("missing < 2.0"), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + } else { + // missing equivalent to null + assertFilterMatches(edf("missing < '2'"), ImmutableList.of()); + assertFilterMatches(edf("missing < 2"), ImmutableList.of()); + assertFilterMatches(edf("missing < 2.0"), ImmutableList.of()); + } + assertFilterMatches(edf("missing > '2'"), ImmutableList.of()); + assertFilterMatches(edf("missing > 2"), ImmutableList.of()); + assertFilterMatches(edf("missing > 2.0"), ImmutableList.of()); + assertFilterMatchesSkipVectorize(edf("like(missing, '1%')"), ImmutableList.of()); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java index 9c40ccb5da1..1bade62c5c8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ExpressionFilterTest.java @@ -40,16 +40,17 @@ import org.apache.druid.math.expr.ExpressionProcessing; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.filter.ExpressionDimFilter; import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.junit.After; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -57,7 +58,6 @@ import java.io.Closeable; import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; @RunWith(Parameterized.class) public class ExpressionFilterTest extends BaseFilterTest @@ -73,27 +73,37 @@ public class ExpressionFilterTest extends BaseFilterTest new LongDimensionSchema("dim1"), new FloatDimensionSchema("dim2"), new StringDimensionSchema("dim3"), - new StringDimensionSchema("dim4") + new StringDimensionSchema("dim4"), + new StringDimensionSchema("dim5") ) ) ) ); - private static final List ROWS = ImmutableList.>of( - ImmutableMap.of("dim0", "0", "dim1", 0L, "dim2", 0.0f, "dim3", "", "dim4", ImmutableList.of("1", "2")), - ImmutableMap.of("dim0", "1", "dim1", 1L, "dim2", 1.0f, "dim3", "10", "dim4", ImmutableList.of()), - ImmutableMap.of("dim0", "2", "dim1", 2L, "dim2", 2.0f, "dim3", "2", "dim4", ImmutableList.of("")), - ImmutableMap.of("dim0", "3", "dim1", 3L, "dim2", 3.0f, "dim3", "1", "dim4", ImmutableList.of("3")), - ImmutableMap.of("dim0", "4", "dim1", 4L, "dim2", 4.0f, "dim3", "1", "dim4", ImmutableList.of("4", "5")), - ImmutableMap.of("dim0", "5", "dim1", 5L, "dim2", 5.0f, "dim3", "5", "dim4", ImmutableList.of("4", "5")), - ImmutableMap.of("dim0", "6", "dim1", 6L, "dim2", 6.0f, "dim3", "1"), - ImmutableMap.of("dim0", "7", "dim1", 7L, "dim2", 7.0f, "dim3", "a"), - ImmutableMap.of("dim0", "8", "dim1", 8L, "dim2", 8.0f, "dim3", 8L), + private static final RowSignature ROW_SIGNATURE = RowSignature.builder() + .add("dim0", ColumnType.STRING) + .add("dim1", ColumnType.LONG) + .add("dim2", ColumnType.FLOAT) + .add("dim3", ColumnType.STRING) + .add("dim4", ColumnType.STRING) + .add("dim5", ColumnType.STRING) + .build(); + + private static final List ROWS = ImmutableList.of( + makeSchemaRow(PARSER, ROW_SIGNATURE, "0", 0L, 0.0f, "", ImmutableList.of("1", "2"), "a"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "1", 1L, 1.0f, "10", ImmutableList.of(), "b"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "2", 2L, 2.0f, "2", ImmutableList.of(""), null), + makeSchemaRow(PARSER, ROW_SIGNATURE, "3", 3L, 3.0f, "1", ImmutableList.of("3"), "c"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "4", 4L, 4.0f, "1", ImmutableList.of("4", "5"), ""), + makeSchemaRow(PARSER, ROW_SIGNATURE, "5", 5L, 5.0f, "5", ImmutableList.of("4", "5"), "d"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "6", 6L, 6.0f, "1", null, "e"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "7", 7L, 7.0f, "a", null, "f"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "8", 8L, 8.0f, 8L, null, "g"), // Note: the "dim3 == 1.234" check in "testOneSingleValuedStringColumn" fails if dim3 is 1.234f instead of 1.234d, // because the literal 1.234 is interpreted as a double, and 1.234f cast to double is not equivalent to 1.234d. - ImmutableMap.of("dim0", "9", "dim1", 9L, "dim2", 9.0f, "dim3", 1.234d, "dim4", 1.234d) - ).stream().map(e -> PARSER.parseBatch(e).get(0)).collect(Collectors.toList()); + makeSchemaRow(PARSER, ROW_SIGNATURE, "9", 9L, 9.0f, 1.234d, 1.234d, null) + ); public ExpressionFilterTest( String testName, @@ -116,9 +126,6 @@ public class ExpressionFilterTest extends BaseFilterTest ); } - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @Before public void setup() { @@ -163,6 +170,32 @@ public class ExpressionFilterTest extends BaseFilterTest assertFilterMatchesSkipVectorize(edf("array_contains(dim3, '1')"), ImmutableList.of("3", "4", "6")); } + @Test + public void testComplement() + { + if (NullHandling.sqlCompatible()) { + assertFilterMatches(edf("dim5 == 'a'"), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(edf("dim5 == 'a'")), + ImmutableList.of("1", "3", "4", "5", "6", "7", "8") + ); + assertFilterMatches( + edf("dim5 == ''"), ImmutableList.of("4") + ); + assertFilterMatches( + NotDimFilter.of(edf("dim5 == ''")), ImmutableList.of("0", "1", "3", "5", "6", "7", "8") + ); + } else { + assertFilterMatches(edf("dim5 == 'a'"), ImmutableList.of("0")); + assertFilterMatches( + NotDimFilter.of(edf("dim5 == 'a'")), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "3", "5", "6", "7", "8") + : ImmutableList.of("1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + } + } + @Test public void testOneMultiValuedStringColumn() { @@ -264,6 +297,34 @@ public class ExpressionFilterTest extends BaseFilterTest assertFilterMatchesSkipVectorize(edf("dim0 == dim4"), ImmutableList.of("3", "4", "5")); } + @Test + public void testNullNotUnknown() + { + assertFilterMatchesSkipVectorize( + edf("isfalse(dim5)"), + NullHandling.sqlCompatible() ? ImmutableList.of("0", "1", "3", "4", "5", "6", "7", "8") : ImmutableList.of("0", "1", "3", "5", "6", "7", "8") + ); + assertFilterMatchesSkipVectorize( + edf("!isfalse(dim5)"), + NullHandling.sqlCompatible() ? ImmutableList.of("2", "9") : ImmutableList.of("2", "4", "9") + ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(edf("isfalse(dim5)")), + NullHandling.sqlCompatible() ? ImmutableList.of("2", "9") : ImmutableList.of("2", "4", "9") + ); + + assertFilterMatchesSkipVectorize( + edf("isfalse(notexist)"), + ImmutableList.of() + ); + assertFilterMatchesSkipVectorize( + edf("!isfalse(notexist)"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(edf("isfalse(notexist)")), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9") + ); + } + @Test public void testMissingColumn() { @@ -279,11 +340,17 @@ public class ExpressionFilterTest extends BaseFilterTest } else { // AS per SQL standard null == null returns false. assertFilterMatches(edf("missing == null"), ImmutableList.of()); + // and inverted too + assertFilterMatches(NotDimFilter.of(edf("missing == null")), ImmutableList.of()); // also this madness doesn't do madness assertFilterMatches( edf("missing == otherMissing"), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of(edf("missing == otherMissing")), + ImmutableList.of() + ); } assertFilterMatches(edf("missing == '1'"), ImmutableList.of()); assertFilterMatches(edf("missing == 2"), ImmutableList.of()); @@ -340,12 +407,14 @@ public class ExpressionFilterTest extends BaseFilterTest Filter filter = edf("dim1 == '1'").toFilter(); Assert.assertFalse(filter.supportsRequiredColumnRewrite()); - expectedException.expect(UnsupportedOperationException.class); - expectedException.expectMessage("Required column rewrite is not supported by this filter."); - filter.rewriteRequiredColumns(ImmutableMap.of("invalidName", "dim1")); + Throwable t = Assert.assertThrows( + UnsupportedOperationException.class, + () -> filter.rewriteRequiredColumns(ImmutableMap.of("invalidName", "dim1")) + ); + Assert.assertEquals("Required column rewrite is not supported by this filter.", t.getMessage()); } - private static ExpressionDimFilter edf(final String expression) + protected static ExpressionDimFilter edf(final String expression) { return new ExpressionDimFilter(expression, null, TestExprMacroTable.INSTANCE); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java index 5867920b4bd..52feb2c91d8 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterCnfConversionTest.java @@ -709,7 +709,7 @@ public class FilterCnfConversionTest return null; } } - ).matches(); + ).matches(false); } private Filter visitSelectorFilters(Filter filter, Function visitAction) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index 3e19b9a113d..42fd7fe45ef 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -163,10 +163,10 @@ public class FilterPartitionTest extends BaseFilterTest private static final List ROWS = ImmutableList.builder() .addAll(DEFAULT_ROWS) - .add(makeDefaultSchemaRow("6", "B453B411", ImmutableList.of("c", "d", "e"), null, null, null, null)) - .add(makeDefaultSchemaRow("7", "HELLO", ImmutableList.of("foo"), null, null, null, null)) - .add(makeDefaultSchemaRow("8", "abc", ImmutableList.of("bar"), null, null, null, null)) - .add(makeDefaultSchemaRow("9", "1", ImmutableList.of("foo", "bar"), null, null, null, null)) + .add(makeDefaultSchemaRow("6", "B453B411", ImmutableList.of("c", "d", "e"), null, null, null, null, null)) + .add(makeDefaultSchemaRow("7", "HELLO", ImmutableList.of("foo"), null, null, null, null, null)) + .add(makeDefaultSchemaRow("8", "abc", ImmutableList.of("bar"), null, null, null, null, null)) + .add(makeDefaultSchemaRow("9", "1", ImmutableList.of("foo", "bar"), null, null, null, null, null)) .build(); public FilterPartitionTest( diff --git a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java index 535a134b035..bb0bf308e19 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/InFilterTest.java @@ -36,15 +36,14 @@ 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.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -58,12 +57,12 @@ import java.util.Map; 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", 10.1, 10.1f, 100L), - makeDefaultSchemaRow("c", "2", ImmutableList.of(""), "2017-05-25", null, 5.5f, 40L), - makeDefaultSchemaRow("d", "1", ImmutableList.of("a"), "2020-01-25", 120.0245, 110.0f, null), - makeDefaultSchemaRow("e", "def", ImmutableList.of("c"), null, 60.0, null, 9001L), - makeDefaultSchemaRow("f", "abc", null, "2020-01-25", 765.432, 123.45f, 12345L) + 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( @@ -77,8 +76,6 @@ public class InFilterTest extends BaseFilterTest super(testName, ROWS, indexBuilder, finisher, cnf, optimize); } - @Rule - public ExpectedException expectedException = ExpectedException.none(); @AfterClass public static void tearDown() throws Exception @@ -93,6 +90,10 @@ public class InFilterTest extends BaseFilterTest toInFilter("dim0"), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of(toInFilter("dim0")), + ImmutableList.of("a", "b", "c", "d", "e", "f") + ); assertFilterMatches( toInFilter("dim0", null), @@ -113,6 +114,11 @@ public class InFilterTest extends BaseFilterTest toInFilter("dim0", "e", "x"), ImmutableList.of("e") ); + + assertFilterMatches( + NotDimFilter.of(toInFilter("dim0", "e", "x")), + ImmutableList.of("a", "b", "c", "d", "f") + ); } @Test @@ -128,22 +134,59 @@ public class InFilterTest extends BaseFilterTest 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") + ); } - - assertFilterMatches( - toInFilter("dim1", "-1", "ab", "de"), - ImmutableList.of() - ); } @Test @@ -215,6 +258,10 @@ public class InFilterTest extends BaseFilterTest 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( @@ -226,17 +273,29 @@ public class InFilterTest extends BaseFilterTest 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"), @@ -263,10 +322,18 @@ public class InFilterTest extends BaseFilterTest 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") @@ -280,10 +347,18 @@ public class InFilterTest extends BaseFilterTest 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") @@ -300,6 +375,10 @@ public class InFilterTest extends BaseFilterTest toInFilterWithFn("dim3", yesNullFn, "NO"), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of(toInFilterWithFn("dim3", yesNullFn, "NO")), + NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("a", "b", "c", "d", "e", "f") + ); assertFilterMatches( toInFilterWithFn("dim3", yesNullFn, "YES"), @@ -361,6 +440,18 @@ public class InFilterTest extends BaseFilterTest 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()); @@ -375,9 +466,39 @@ public class InFilterTest extends BaseFilterTest 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")); @@ -400,9 +521,8 @@ public class InFilterTest extends BaseFilterTest Filter rewrittenFilter = filter.rewriteRequiredColumns(ImmutableMap.of("dim0", "dim1")); Assert.assertEquals(filter2, rewrittenFilter); - expectedException.expect(IAE.class); - expectedException.expectMessage("Received a non-applicable rewrite: {invalidName=dim1}, filter's dimension: dim0"); - filter.rewriteRequiredColumns(ImmutableMap.of("invalidName", "dim1")); + 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 @@ -425,7 +545,8 @@ public class InFilterTest extends BaseFilterTest "longPredicateSupplier", "floatPredicateSupplier", "doublePredicateSupplier", - "stringPredicateSupplier" + "stringPredicateSupplier", + "hasNull" ) .verify(); } diff --git a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java index eb81334acb5..7ff919fc948 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/LikeFilterTest.java @@ -36,13 +36,14 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.extraction.SubstringDimExtractionFn; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.LikeDimFilter; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.StorageAdapter; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; import org.junit.AfterClass; import org.junit.Assert; -import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -58,18 +59,25 @@ public class LikeFilterTest extends BaseFilterTest private static final InputRowParser> PARSER = new MapInputRowParser( new TimeAndDimsParseSpec( new TimestampSpec(TIMESTAMP_COLUMN, "iso", DateTimes.of("2000")), - DimensionsSpec.EMPTY + DimensionsSpec.builder() + .setDimensions(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2"))) + .build() ) ); + private static final RowSignature ROW_SIGNATURE = RowSignature.builder() + .add("dim0", ColumnType.STRING) + .add("dim1", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .build(); private static final List ROWS = ImmutableList.of( - PARSER.parseBatch(ImmutableMap.of("dim0", "0", "dim1", "")).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "1", "dim1", "foo")).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "2", "dim1", "foobar")).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "3", "dim1", "bar")).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "4", "dim1", "foobarbaz")).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "5", "dim1", "foo%bar")).get(0), - PARSER.parseBatch(ImmutableMap.of("dim0", "6", "dim1", "new\nline")).get(0) + makeSchemaRow(PARSER, ROW_SIGNATURE, "0", "", ""), + makeSchemaRow(PARSER, ROW_SIGNATURE, "1", "foo", "aaa"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "2", "foobar", "aab"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "3", "bar", null), + makeSchemaRow(PARSER, ROW_SIGNATURE, "4", "foobarbaz", "abb"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "5", "foo%bar", "bbb"), + makeSchemaRow(PARSER, ROW_SIGNATURE, "6", "new\nline", "bbz") ); public LikeFilterTest( @@ -83,9 +91,6 @@ public class LikeFilterTest extends BaseFilterTest super(testName, ROWS, indexBuilder, finisher, cnf, optimize); } - @Rule - public ExpectedException expectedException = ExpectedException.none(); - @AfterClass public static void tearDown() throws Exception { @@ -99,6 +104,21 @@ public class LikeFilterTest extends BaseFilterTest new LikeDimFilter("dim1", "bar", null, null), ImmutableList.of("3") ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim1", "bar", null, null)), + ImmutableList.of("0", "1", "2", "4", "5", "6") + ); + + assertFilterMatches( + new LikeDimFilter("dim2", "bbb", null, null), + ImmutableList.of("5") + ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim2", "bbb", null, null)), + NullHandling.replaceWithDefault() + ? ImmutableList.of("0", "1", "2", "3", "4", "6") + : ImmutableList.of("0", "1", "2", "4", "6") + ); } @Test @@ -117,6 +137,21 @@ public class LikeFilterTest extends BaseFilterTest new LikeDimFilter("dim1", "bar", null, new SubstringDimExtractionFn(3, 3)), ImmutableList.of("2", "4") ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim1", "bar", null, new SubstringDimExtractionFn(3, 3))), + ImmutableList.of("0", "1", "3", "5", "6") + ); + + assertFilterMatches( + new LikeDimFilter("dim2", "bbb", null, new SubstringDimExtractionFn(0, 3)), + ImmutableList.of("5") + ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim2", "bbb", null, new SubstringDimExtractionFn(0, 3))), + NullHandling.replaceWithDefault() + ? ImmutableList.of("0", "1", "2", "3", "4", "6") + : ImmutableList.of("0", "1", "2", "4", "6") + ); } @Test @@ -126,6 +161,19 @@ public class LikeFilterTest extends BaseFilterTest new LikeDimFilter("dim1", "foo%", null, null), ImmutableList.of("1", "2", "4", "5") ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim1", "foo%", null, null)), + ImmutableList.of("0", "3", "6") + ); + + assertFilterMatches( + new LikeDimFilter("dim2", "aa%", null, null), + ImmutableList.of("1", "2") + ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim2", "aa%", null, null)), + NullHandling.replaceWithDefault() ? ImmutableList.of("0", "3", "4", "5", "6") : ImmutableList.of("0", "4", "5", "6") + ); } @Test @@ -144,6 +192,21 @@ public class LikeFilterTest extends BaseFilterTest new LikeDimFilter("dim1", "a%", null, new SubstringDimExtractionFn(1, null)), ImmutableList.of("3") ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim1", "a%", null, new SubstringDimExtractionFn(1, null))), + ImmutableList.of("0", "1", "2", "4", "5", "6") + ); + + assertFilterMatches( + new LikeDimFilter("dim2", "a%", null, new SubstringDimExtractionFn(1, null)), + ImmutableList.of("1", "2") + ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim2", "a%", null, new SubstringDimExtractionFn(1, null))), + NullHandling.replaceWithDefault() + ? ImmutableList.of("0", "3", "4", "5", "6") + : ImmutableList.of("0", "4", "5", "6") + ); } @Test @@ -153,6 +216,19 @@ public class LikeFilterTest extends BaseFilterTest new LikeDimFilter("dim1", "%oba%", null, null), ImmutableList.of("2", "4") ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim1", "%oba%", null, null)), + ImmutableList.of("0", "1", "3", "5", "6") + ); + + assertFilterMatches( + new LikeDimFilter("dim2", "%b%", null, null), + ImmutableList.of("2", "4", "5", "6") + ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim2", "%b%", null, null)), + NullHandling.sqlCompatible() ? ImmutableList.of("0", "1") : ImmutableList.of("0", "1", "3") + ); } @Test @@ -196,6 +272,22 @@ public class LikeFilterTest extends BaseFilterTest new LikeDimFilter("dim1", "%", "@", null), ImmutableList.of("0", "1", "2", "3", "4", "5", "6") ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim1", "%", "@", null)), + ImmutableList.of() + ); + + // doesnt match null tho in sql compatible mode + assertFilterMatches( + new LikeDimFilter("dim2", "%", "@", null), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "2", "4", "5", "6") + : ImmutableList.of("0", "1", "2", "3", "4", "5", "6") + ); + assertFilterMatches( + NotDimFilter.of(new LikeDimFilter("dim2", "%", "@", null)), + ImmutableList.of() + ); } @Test @@ -301,9 +393,8 @@ public class LikeFilterTest extends BaseFilterTest Filter rewrittenFilter = filter.rewriteRequiredColumns(ImmutableMap.of("dim0", "dim1")); Assert.assertEquals(filter2, rewrittenFilter); - expectedException.expect(IAE.class); - expectedException.expectMessage("Received a non-applicable rewrite: {invalidName=dim1}, filter's dimension: dim0"); - filter.rewriteRequiredColumns(ImmutableMap.of("invalidName", "dim1")); + 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 diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTest.java b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTest.java deleted file mode 100644 index 54fcb7696e6..00000000000 --- a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTest.java +++ /dev/null @@ -1,237 +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.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import nl.jqno.equalsverifier.EqualsVerifier; -import org.apache.druid.common.config.NullHandling; -import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.java.util.common.Pair; -import org.apache.druid.query.filter.FilterTuning; -import org.apache.druid.query.filter.NullFilter; -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.Arrays; - -@RunWith(Parameterized.class) -public class NullFilterTest extends BaseFilterTest -{ - public NullFilterTest( - String testName, - IndexBuilder indexBuilder, - Function> finisher, - boolean cnf, - boolean optimize - ) - { - super(testName, DEFAULT_ROWS, indexBuilder, finisher, cnf, optimize); - } - - @AfterClass - public static void tearDown() throws Exception - { - BaseFilterTest.tearDown(NullFilterTest.class.getName()); - } - - @Test - public void testSingleValueStringColumnWithoutNulls() - { - assertFilterMatches(NullFilter.forColumn("dim0"), ImmutableList.of()); - } - - @Test - public void testSingleValueVirtualStringColumnWithoutNulls() - { - assertFilterMatches(NullFilter.forColumn("vdim0"), ImmutableList.of()); - } - - @Test - public void testListFilteredVirtualColumn() - { - assertFilterMatchesSkipVectorize(NullFilter.forColumn("allow-dim0"), ImmutableList.of("0", "1", "2", "5")); - assertFilterMatchesSkipVectorize(NullFilter.forColumn("deny-dim0"), ImmutableList.of("3", "4")); - if (isAutoSchema()) { - return; - } - assertFilterMatchesSkipVectorize(NullFilter.forColumn("allow-dim2"), ImmutableList.of("1", "2", "4", "5")); - if (NullHandling.replaceWithDefault()) { - assertFilterMatchesSkipVectorize( - NullFilter.forColumn("deny-dim2"), - ImmutableList.of("1", "2", "3", "5") - ); - } else { - assertFilterMatchesSkipVectorize( - NullFilter.forColumn("deny-dim2"), - ImmutableList.of("1", "3", "5") - ); - } - } - - @Test - public void testSingleValueStringColumnWithNulls() - { - // testSingleValueStringColumnWithoutNulls but with virtual column selector - if (NullHandling.replaceWithDefault()) { - assertFilterMatches(NullFilter.forColumn("dim1"), ImmutableList.of("0")); - } else { - assertFilterMatches(NullFilter.forColumn("dim1"), ImmutableList.of()); - } - } - - @Test - public void testSingleValueVirtualStringColumnWithNulls() - { - // testSingleValueStringColumnWithNulls but with virtual column selector - if (NullHandling.replaceWithDefault()) { - assertFilterMatches(NullFilter.forColumn("vdim1"), ImmutableList.of("0")); - } else { - assertFilterMatches(NullFilter.forColumn("vdim1"), ImmutableList.of()); - } - } - - @Test - public void testMultiValueStringColumn() - { - if (NullHandling.replaceWithDefault()) { - if (isAutoSchema()) { - assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("5")); - } else { - assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("1", "2", "5")); - } - } else { - // only one array row is totally null - if (isAutoSchema()) { - assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("5")); - } else { - assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("1", "5")); - } - } - } - - @Test - public void testMissingColumnSpecifiedInDimensionList() - { - assertFilterMatches(NullFilter.forColumn("dim3"), ImmutableList.of("0", "1", "2", "3", "4", "5")); - } - - @Test - public void testMissingColumnNotSpecifiedInDimensionList() - { - assertFilterMatches(NullFilter.forColumn("dim4"), ImmutableList.of("0", "1", "2", "3", "4", "5")); - } - - - @Test - public void testVirtualNumericColumnNullsAndDefaults() - { - if (canTestNumericNullsAsDefaultValues) { - assertFilterMatches(NullFilter.forColumn("vf0"), ImmutableList.of()); - assertFilterMatches(NullFilter.forColumn("vd0"), ImmutableList.of()); - assertFilterMatches(NullFilter.forColumn("vl0"), ImmutableList.of()); - } else { - assertFilterMatches(NullFilter.forColumn("vf0"), ImmutableList.of("4")); - assertFilterMatches(NullFilter.forColumn("vd0"), ImmutableList.of("2")); - assertFilterMatches(NullFilter.forColumn("vl0"), ImmutableList.of("3")); - } - } - - @Test - public void testNumericColumnNullsAndDefaults() - { - if (canTestNumericNullsAsDefaultValues) { - assertFilterMatches(NullFilter.forColumn("f0"), ImmutableList.of()); - assertFilterMatches(NullFilter.forColumn("d0"), ImmutableList.of()); - assertFilterMatches(NullFilter.forColumn("l0"), ImmutableList.of()); - } else { - assertFilterMatches(NullFilter.forColumn("f0"), ImmutableList.of("4")); - assertFilterMatches(NullFilter.forColumn("d0"), ImmutableList.of("2")); - assertFilterMatches(NullFilter.forColumn("l0"), ImmutableList.of("3")); - } - } - - @Test - public void testArrays() - { - if (isAutoSchema()) { - // only auto schema ingests arrays - /* - dim0 .. arrayString arrayLong arrayDouble - "0", .. ["a", "b", "c"], [1L, 2L, 3L], [1.1, 2.2, 3.3] - "1", .. [], [], [1.1, 2.2, 3.3] - "2", .. null, [1L, 2L, 3L], [null] - "3", .. ["a", "b", "c"], null, [] - "4", .. ["c", "d"], [null], [-1.1, -333.3] - "5", .. [null], [123L, 345L], null - */ - assertFilterMatches( - new NullFilter("arrayString", null), - ImmutableList.of("2") - ); - assertFilterMatches( - new NullFilter("arrayLong", null), - ImmutableList.of("3") - ); - assertFilterMatches( - new NullFilter("arrayDouble", null), - ImmutableList.of("5") - ); - } - } - - @Test - public void testSerde() throws JsonProcessingException - { - ObjectMapper mapper = new DefaultObjectMapper(); - NullFilter filter = new NullFilter("x", null); - String s = mapper.writeValueAsString(filter); - Assert.assertEquals(filter, mapper.readValue(s, NullFilter.class)); - } - - @Test - public void testGetCacheKey() - { - NullFilter f1 = new NullFilter("x", null); - NullFilter f1_2 = new NullFilter("x", null); - NullFilter f2 = new NullFilter("y", null); - NullFilter f3 = new NullFilter("x", new FilterTuning(true, 1234, null)); - Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey()); - Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey())); - Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey()); - } - - @Test - public void test_equals() - { - EqualsVerifier.forClass(NullFilter.class).usingGetClass() - .withNonnullFields("column") - .withIgnoredFields("cachedOptimizedFilter") - .verify(); - } -} diff --git a/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java new file mode 100644 index 00000000000..b123ead53a5 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/segment/filter/NullFilterTests.java @@ -0,0 +1,343 @@ +/* + * 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.collect.ImmutableList; +import nl.jqno.equalsverifier.EqualsVerifier; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.filter.FilterTuning; +import org.apache.druid.query.filter.NotDimFilter; +import org.apache.druid.query.filter.NullFilter; +import org.apache.druid.segment.IndexBuilder; +import org.apache.druid.segment.StorageAdapter; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.Closeable; +import java.util.Arrays; + +@RunWith(Enclosed.class) +public class NullFilterTests +{ + @RunWith(Parameterized.class) + public static class NullFilterTest extends BaseFilterTest + { + public NullFilterTest( + String testName, + IndexBuilder indexBuilder, + Function> finisher, + boolean cnf, + boolean optimize + ) + { + super(testName, DEFAULT_ROWS, indexBuilder, finisher, cnf, optimize); + } + + @AfterClass + public static void tearDown() throws Exception + { + BaseFilterTest.tearDown(NullFilterTest.class.getName()); + } + + @Test + public void testSingleValueStringColumnWithoutNulls() + { + assertFilterMatches(NullFilter.forColumn("dim0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("dim0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + } + + @Test + public void testSingleValueVirtualStringColumnWithoutNulls() + { + assertFilterMatches(NullFilter.forColumn("vdim0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("vdim0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + } + + @Test + public void testListFilteredVirtualColumn() + { + assertFilterMatchesSkipVectorize(NullFilter.forColumn("allow-dim0"), ImmutableList.of("0", "1", "2", "5")); + assertFilterMatchesSkipVectorize(NotDimFilter.of(NullFilter.forColumn("allow-dim0")), ImmutableList.of("3", "4")); + assertFilterMatchesSkipVectorize(NullFilter.forColumn("deny-dim0"), ImmutableList.of("3", "4")); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(NullFilter.forColumn("deny-dim0")), + ImmutableList.of("0", "1", "2", "5") + ); + if (isAutoSchema()) { + return; + } + assertFilterMatchesSkipVectorize(NullFilter.forColumn("allow-dim2"), ImmutableList.of("1", "2", "4", "5")); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(NullFilter.forColumn("allow-dim2")), + ImmutableList.of("0", "3") + ); + if (NullHandling.replaceWithDefault()) { + assertFilterMatchesSkipVectorize( + NullFilter.forColumn("deny-dim2"), + ImmutableList.of("1", "2", "3", "5") + ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(NullFilter.forColumn("deny-dim2")), + ImmutableList.of("0", "4") + ); + } else { + assertFilterMatchesSkipVectorize( + NullFilter.forColumn("deny-dim2"), + ImmutableList.of("1", "3", "5") + ); + assertFilterMatchesSkipVectorize( + NotDimFilter.of(NullFilter.forColumn("deny-dim2")), + ImmutableList.of("0", "2", "4") + ); + } + } + + @Test + public void testSingleValueStringColumnWithNulls() + { + // testSingleValueStringColumnWithoutNulls but with virtual column selector + if (NullHandling.replaceWithDefault()) { + assertFilterMatches(NullFilter.forColumn("dim1"), ImmutableList.of("0")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("dim1")), ImmutableList.of("1", "2", "3", "4", "5")); + + assertFilterMatches(NullFilter.forColumn("s0"), ImmutableList.of("0", "3")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("s0")), ImmutableList.of("1", "2", "4", "5")); + } else { + assertFilterMatches(NullFilter.forColumn("dim1"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("dim1")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("s0"), ImmutableList.of("3")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("s0")), ImmutableList.of("0", "1", "2", "4", "5")); + } + } + + @Test + public void testSingleValueVirtualStringColumnWithNulls() + { + // testSingleValueStringColumnWithNulls but with virtual column selector + if (NullHandling.replaceWithDefault()) { + assertFilterMatches(NullFilter.forColumn("vdim1"), ImmutableList.of("0")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("vdim1")), ImmutableList.of("1", "2", "3", "4", "5")); + + assertFilterMatches(NullFilter.forColumn("vs0"), ImmutableList.of("0", "3")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("vs0")), ImmutableList.of("1", "2", "4", "5")); + } else { + assertFilterMatches(NullFilter.forColumn("vdim1"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("vdim1")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("vs0"), ImmutableList.of("3")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("vs0")), ImmutableList.of("0", "1", "2", "4", "5")); + } + } + + @Test + public void testMultiValueStringColumn() + { + if (NullHandling.replaceWithDefault()) { + if (isAutoSchema()) { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("5")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("dim2")), ImmutableList.of("0", "1", "2", "3", "4")); + } else { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("1", "2", "5")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("dim2")), ImmutableList.of("0", "3", "4")); + } + } else { + // only one array row is totally null + if (isAutoSchema()) { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("5")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("dim2")), ImmutableList.of("0", "1", "2", "3", "4")); + } else { + assertFilterMatches(NullFilter.forColumn("dim2"), ImmutableList.of("1", "5")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("dim2")), ImmutableList.of("0", "2", "3", "4")); + } + } + } + + @Test + public void testMissingColumnSpecifiedInDimensionList() + { + assertFilterMatches(NullFilter.forColumn("dim3"), ImmutableList.of("0", "1", "2", "3", "4", "5")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("dim3")), ImmutableList.of()); + } + + @Test + public void testMissingColumnNotSpecifiedInDimensionList() + { + assertFilterMatches(NullFilter.forColumn("dim4"), ImmutableList.of("0", "1", "2", "3", "4", "5")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("dim4")), ImmutableList.of()); + } + + + @Test + public void testVirtualNumericColumnNullsAndDefaults() + { + if (canTestNumericNullsAsDefaultValues) { + assertFilterMatches(NullFilter.forColumn("vf0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("vf0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("vd0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("vd0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("vl0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("vl0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + } else { + assertFilterMatches(NullFilter.forColumn("vf0"), ImmutableList.of("4")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("vf0")), ImmutableList.of("0", "1", "2", "3", "5")); + + assertFilterMatches(NullFilter.forColumn("vd0"), ImmutableList.of("2")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("vd0")), ImmutableList.of("0", "1", "3", "4", "5")); + + assertFilterMatches(NullFilter.forColumn("vl0"), ImmutableList.of("3")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("vl0")), ImmutableList.of("0", "1", "2", "4", "5")); + } + } + + @Test + public void testNumericColumnNullsAndDefaults() + { + if (canTestNumericNullsAsDefaultValues) { + assertFilterMatches(NullFilter.forColumn("f0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("f0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("d0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("d0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("l0"), ImmutableList.of()); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("l0")), + ImmutableList.of("0", "1", "2", "3", "4", "5") + ); + + } else { + assertFilterMatches(NullFilter.forColumn("f0"), ImmutableList.of("4")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("f0")), ImmutableList.of("0", "1", "2", "3", "5")); + + assertFilterMatches(NullFilter.forColumn("d0"), ImmutableList.of("2")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("d0")), ImmutableList.of("0", "1", "3", "4", "5")); + + assertFilterMatches(NullFilter.forColumn("l0"), ImmutableList.of("3")); + assertFilterMatches(NotDimFilter.of(NullFilter.forColumn("l0")), ImmutableList.of("0", "1", "2", "4", "5")); + } + } + + @Test + public void testArrays() + { + if (isAutoSchema()) { + // only auto schema ingests arrays + /* + dim0 .. arrayString arrayLong arrayDouble + "0", .. ["a", "b", "c"], [1L, 2L, 3L], [1.1, 2.2, 3.3] + "1", .. [], [], [1.1, 2.2, 3.3] + "2", .. null, [1L, 2L, 3L], [null] + "3", .. ["a", "b", "c"], null, [] + "4", .. ["c", "d"], [null], [-1.1, -333.3] + "5", .. [null], [123L, 345L], null + */ + assertFilterMatches(NullFilter.forColumn("arrayString"), ImmutableList.of("2")); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("arrayString")), + ImmutableList.of("0", "1", "3", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("arrayLong"), ImmutableList.of("3")); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("arrayLong")), + ImmutableList.of("0", "1", "2", "4", "5") + ); + + assertFilterMatches(NullFilter.forColumn("arrayDouble"), ImmutableList.of("5")); + assertFilterMatches( + NotDimFilter.of(NullFilter.forColumn("arrayDouble")), + ImmutableList.of("0", "1", "2", "3", "4") + ); + } + } + } + + public static class NullFilterNonParameterizedTest + { + @Test + public void testSerde() throws JsonProcessingException + { + ObjectMapper mapper = new DefaultObjectMapper(); + NullFilter filter = new NullFilter("x", null); + String s = mapper.writeValueAsString(filter); + Assert.assertEquals(filter, mapper.readValue(s, NullFilter.class)); + } + + @Test + public void testGetCacheKey() + { + NullFilter f1 = new NullFilter("x", null); + NullFilter f1_2 = new NullFilter("x", null); + NullFilter f2 = new NullFilter("y", null); + NullFilter f3 = new NullFilter("x", new FilterTuning(true, 1234, null)); + Assert.assertArrayEquals(f1.getCacheKey(), f1_2.getCacheKey()); + Assert.assertFalse(Arrays.equals(f1.getCacheKey(), f2.getCacheKey())); + Assert.assertArrayEquals(f1.getCacheKey(), f3.getCacheKey()); + } + + @Test + public void test_equals() + { + EqualsVerifier.forClass(NullFilter.class).usingGetClass() + .withNonnullFields("column") + .withIgnoredFields("cachedOptimizedFilter") + .verify(); + } + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java index 3a7418421f5..30ec09d0d3e 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/PredicateValueMatcherFactoryTest.java @@ -53,14 +53,14 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes public void testDimensionProcessorSingleValuedDimensionMatchingValue() { final ValueMatcher matcher = forSelector("0").makeDimensionProcessor(DimensionSelector.constant("0"), false); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test public void testDimensionProcessorSingleValuedDimensionNotMatchingValue() { final ValueMatcher matcher = forSelector("1").makeDimensionProcessor(DimensionSelector.constant("0"), false); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -81,7 +81,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); final ValueMatcher matcher = forSelector("v2") .makeDimensionProcessor(columnSupplier.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), true); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -102,7 +102,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); final ValueMatcher matcher = forSelector("v3") .makeDimensionProcessor(columnSupplier.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), true); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -115,7 +115,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("2.f").makeFloatProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -128,7 +128,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("5.f").makeFloatProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -141,7 +141,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("2.").makeDoubleProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -154,7 +154,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("5.").makeDoubleProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -167,7 +167,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("2").makeLongProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -180,7 +180,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("5").makeLongProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -193,7 +193,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector(null).makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -207,9 +207,9 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes columnValueSelector.advance(); final ValueMatcher matcher = forSelector(null).makeComplexProcessor(columnValueSelector); if (NullHandling.sqlCompatible()) { - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } else { - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } } @@ -223,7 +223,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -236,7 +236,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -249,7 +249,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -262,7 +262,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -275,7 +275,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.f").makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -288,7 +288,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.f").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -301,7 +301,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.d").makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -314,7 +314,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("11.d").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -327,7 +327,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -340,7 +340,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -353,7 +353,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -366,7 +366,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -379,7 +379,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector(null).makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -392,7 +392,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("false").makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -405,7 +405,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("false").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } @Test @@ -419,7 +419,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes columnValueSelector.advance(); final String base64Encoded = StringUtils.encodeBase64String(StringUtils.toUtf8("var")); final ValueMatcher matcher = forSelector(base64Encoded).makeComplexProcessor(columnValueSelector); - Assert.assertTrue(matcher.matches()); + Assert.assertTrue(matcher.matches(false)); } @Test @@ -432,7 +432,7 @@ public class PredicateValueMatcherFactoryTest extends InitializedNullHandlingTes ); columnValueSelector.advance(); final ValueMatcher matcher = forSelector("val").makeComplexProcessor(columnValueSelector); - Assert.assertFalse(matcher.matches()); + Assert.assertFalse(matcher.matches(false)); } private static PredicateValueMatcherFactory forSelector(@Nullable String value) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java index 4ac578afce8..132d6af64db 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/RangeFilterTests.java @@ -39,6 +39,7 @@ import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; +import org.apache.druid.query.filter.NotDimFilter; import org.apache.druid.query.filter.RangeFilter; import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.StorageAdapter; @@ -70,6 +71,7 @@ public class RangeFilterTests "-1000", ImmutableList.of("a"), null, + "d", 6.6, null, 10L, @@ -82,6 +84,7 @@ public class RangeFilterTests "-10.012", ImmutableList.of("d"), null, + "e", null, 3.0f, null, @@ -130,6 +133,30 @@ public class RangeFilterTests new RangeFilter("vdim0", ColumnType.STRING, null, "z", false, false, null), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("dim1", ColumnType.STRING, null, "z", false, false, null)), + ImmutableList.of() + ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("vdim0", ColumnType.STRING, null, "z", false, false, null)), + ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("s0", ColumnType.STRING, null, "b", false, false, null), + ImmutableList.of("0", "1", "2", "5") + ); + assertFilterMatches( + new RangeFilter("vs0", ColumnType.STRING, null, "b", false, false, null), + ImmutableList.of("0", "1", "2", "5") + ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("s0", ColumnType.STRING, null, "b", false, false, null)), + ImmutableList.of("4", "6", "7") + ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("vs0", ColumnType.STRING, null, "b", false, false, null)), + ImmutableList.of("4", "6", "7") + ); } else { assertFilterMatches( new RangeFilter("dim1", ColumnType.STRING, null, "z", false, false, null), @@ -139,10 +166,38 @@ public class RangeFilterTests new RangeFilter("vdim0", ColumnType.STRING, null, "z", false, false, null), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("dim1", ColumnType.STRING, null, "z", false, false, null)), + NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("0") + ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("vdim0", ColumnType.STRING, null, "z", false, false, null)), + NullHandling.sqlCompatible() ? ImmutableList.of("0") : ImmutableList.of() + ); + assertFilterMatches( + new RangeFilter("s0", ColumnType.STRING, null, "b", false, false, null), + ImmutableList.of("1", "2", "5") + ); + assertFilterMatches( + new RangeFilter("vs0", ColumnType.STRING, null, "b", false, false, null), + ImmutableList.of("1", "2", "5") + ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("s0", ColumnType.STRING, null, "b", false, false, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("4", "6", "7") + : ImmutableList.of("0", "3", "4", "6", "7") + ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("vs0", ColumnType.STRING, null, "b", false, false, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("4", "6", "7") + : ImmutableList.of("0", "3", "4", "6", "7") + ); } if (isAutoSchema()) { - // auto schema ingests arrays instead of mvds.. this filter doesn't currently support arrays + // auto schema ingests arrays instead of mvds.. this is covered in array tests } else { assertFilterMatches( new RangeFilter("dim2", ColumnType.STRING, null, "z", false, false, null), @@ -269,6 +324,10 @@ public class RangeFilterTests new RangeFilter("dim3", ColumnType.STRING, null, "z", false, true, null), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("dim3", ColumnType.STRING, null, "z", false, true, null)), + NullHandling.sqlCompatible() ? ImmutableList.of() : ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); } @@ -287,6 +346,10 @@ public class RangeFilterTests new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", false, true, null), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", false, true, null)), + ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + ); } @Test @@ -296,6 +359,10 @@ public class RangeFilterTests new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", false, false, null), ImmutableList.of("5") ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("dim1", ColumnType.STRING, "abc", "abc", false, false, null)), + ImmutableList.of("0", "1", "2", "3", "4", "6", "7") + ); } @Test @@ -367,7 +434,12 @@ public class RangeFilterTests new RangeFilter("d0", ColumnType.DOUBLE, 2L, 3L, false, true, null), ImmutableList.of() ); - + assertFilterMatches( + NotDimFilter.of(new RangeFilter("d0", ColumnType.DOUBLE, 2L, 3L, false, true, null)), + NullHandling.replaceWithDefault() + ? ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + : ImmutableList.of("0", "1", "3", "4", "5", "6") + ); assertFilterMatches( new RangeFilter("f0", ColumnType.LONG, 2L, 3L, false, true, null), ImmutableList.of() @@ -376,6 +448,12 @@ public class RangeFilterTests new RangeFilter("f0", ColumnType.DOUBLE, 2L, 3L, false, true, null), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("f0", ColumnType.DOUBLE, 2L, 3L, false, true, null)), + NullHandling.replaceWithDefault() + ? ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + : ImmutableList.of("0", "1", "2", "3", "5", "7") + ); assertFilterMatches( new RangeFilter("l0", ColumnType.LONG, 2L, 3L, false, true, null), ImmutableList.of() @@ -384,6 +462,12 @@ public class RangeFilterTests new RangeFilter("l0", ColumnType.DOUBLE, 2L, 3L, false, true, null), ImmutableList.of() ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("l0", ColumnType.DOUBLE, 2L, 3L, false, true, null)), + NullHandling.replaceWithDefault() + ? ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7") + : ImmutableList.of("0", "1", "2", "4", "5", "6") + ); } @Test @@ -432,6 +516,12 @@ public class RangeFilterTests new RangeFilter("d0", ColumnType.DOUBLE, 120.0245, 120.0245, false, false, null), ImmutableList.of("3") ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("d0", ColumnType.DOUBLE, 120.0245, 120.0245, false, false, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "4", "5", "6") + : ImmutableList.of("0", "1", "2", "4", "5", "6", "7") + ); assertFilterMatches( new RangeFilter("d0", ColumnType.FLOAT, 120.0245f, 120.0245f, false, false, null), ImmutableList.of() @@ -448,10 +538,22 @@ public class RangeFilterTests new RangeFilter("f0", ColumnType.FLOAT, 10.1f, 10.1f, false, false, null), ImmutableList.of("1") ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("f0", ColumnType.FLOAT, 10.1f, 10.1f, false, false, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "2", "3", "5", "7") + : ImmutableList.of("0", "2", "3", "4", "5", "6", "7") + ); assertFilterMatches( new RangeFilter("l0", ColumnType.LONG, 12345L, 12345L, false, false, null), ImmutableList.of("5") ); + assertFilterMatches( + NotDimFilter.of(new RangeFilter("l0", ColumnType.LONG, 12345L, 12345L, false, false, null)), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "2", "4", "6") + : ImmutableList.of("0", "1", "2", "3", "4", "6", "7") + ); assertFilterMatches( new RangeFilter("l0", ColumnType.DOUBLE, 12345.0, 12345.0, false, false, null), ImmutableList.of("5") @@ -907,6 +1009,22 @@ public class RangeFilterTests ), ImmutableList.of("0", "3") ); + assertFilterMatches( + NotDimFilter.of( + new RangeFilter( + "arrayString", + ColumnType.STRING_ARRAY, + new Object[]{"a", "b", "c"}, + new Object[]{"a", "b", "c"}, + false, + false, + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("1", "4", "5", "6", "7") + : ImmutableList.of("1", "2", "4", "5", "6", "7") + ); assertFilterMatches( new RangeFilter( "arrayString", @@ -1010,6 +1128,22 @@ public class RangeFilterTests ), ImmutableList.of("1") ); + assertFilterMatches( + NotDimFilter.of( + new RangeFilter( + "arrayLong", + ColumnType.LONG_ARRAY, + null, + new Object[]{}, + false, + false, + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "2", "4", "5", "6", "7") + : ImmutableList.of("0", "2", "3", "4", "5", "6", "7") + ); assertFilterMatches( new RangeFilter( @@ -1103,6 +1237,22 @@ public class RangeFilterTests ), ImmutableList.of("3") ); + assertFilterMatches( + NotDimFilter.of( + new RangeFilter( + "arrayDouble", + ColumnType.DOUBLE_ARRAY, + null, + new Object[]{}, + false, + false, + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "1", "2", "4", "6", "7") + : ImmutableList.of("0", "1", "2", "4", "5", "6", "7") + ); assertFilterMatches( new RangeFilter( @@ -1390,6 +1540,22 @@ public class RangeFilterTests ), ImmutableList.of("1", "2", "5") ); + assertFilterMatches( + NotDimFilter.of( + new RangeFilter( + "variant", + ColumnType.LONG, + 100L, + null, + false, + false, + null + ) + ), + NullHandling.sqlCompatible() + ? ImmutableList.of("0", "3", "4") + : ImmutableList.of("0", "3", "4", "6", "7") + ); // lexicographical comparison assertFilterMatches( new RangeFilter( diff --git a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java index 0f450a60117..ed355b03e9a 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/ValueMatchersTest.java @@ -81,38 +81,38 @@ public class ValueMatchersTest extends InitializedNullHandlingTest @Test public void testNullDimensionSelectorCanBeBoolean() { - Boolean resultMatchNull = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNull = ValueMatchers.toConstantMatcherTypeIfPossible( DimensionSelector.constant(null), false, string -> string == null ); Assert.assertNotNull(resultMatchNull); - Assert.assertTrue(resultMatchNull); + Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNull); - Boolean resultMatchNotNull = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNotNull = ValueMatchers.toConstantMatcherTypeIfPossible( DimensionSelector.constant(null), false, string -> string != null ); Assert.assertNotNull(resultMatchNotNull); - Assert.assertFalse(resultMatchNotNull); + Assert.assertEquals(ConstantMatcherType.ALL_UNKNOWN, resultMatchNotNull); - Boolean resultMatchNonNilConstant = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNonNilConstant = ValueMatchers.toConstantMatcherTypeIfPossible( supplierSingleConstant.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), false, string -> string != null ); Assert.assertNotNull(resultMatchNonNilConstant); - Assert.assertTrue(resultMatchNonNilConstant); + Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNonNilConstant); - Boolean resultMatchNonNil = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNonNil = ValueMatchers.toConstantMatcherTypeIfPossible( supplierSingle.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), false, string -> string != null ); Assert.assertNull(resultMatchNonNil); - Boolean resultMatchNonNilMulti = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNonNilMulti = ValueMatchers.toConstantMatcherTypeIfPossible( supplierMulti.get().makeDimensionSelector(new SimpleAscendingOffset(1), null), true, string -> string != null @@ -123,38 +123,38 @@ public class ValueMatchersTest extends InitializedNullHandlingTest @Test public void testNilVectorSelectorCanBeBoolean() { - Boolean resultMatchNull = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNull = ValueMatchers.toConstantMatcherTypeIfPossible( NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)), false, string -> string == null ); Assert.assertNotNull(resultMatchNull); - Assert.assertTrue(resultMatchNull); + Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNull); - Boolean resultMatchNotNull = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNotNull = ValueMatchers.toConstantMatcherTypeIfPossible( NilVectorSelector.create(new NoFilterVectorOffset(10, 0, 100)), false, string -> string != null ); Assert.assertNotNull(resultMatchNotNull); - Assert.assertFalse(resultMatchNotNull); + Assert.assertEquals(ConstantMatcherType.ALL_UNKNOWN, resultMatchNotNull); - Boolean resultMatchNotNilConstant = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNotNilConstant = ValueMatchers.toConstantMatcherTypeIfPossible( supplierSingleConstant.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), false, string -> string != null ); Assert.assertNotNull(resultMatchNotNilConstant); - Assert.assertTrue(resultMatchNotNilConstant); + Assert.assertEquals(ConstantMatcherType.ALL_TRUE, resultMatchNotNilConstant); - Boolean resultMatchNotNil = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNotNil = ValueMatchers.toConstantMatcherTypeIfPossible( supplierSingle.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), false, string -> string != null ); Assert.assertNull(resultMatchNotNil); - Boolean resultMatchNotNilMulti = ValueMatchers.toBooleanIfPossible( + ConstantMatcherType resultMatchNotNilMulti = ValueMatchers.toConstantMatcherTypeIfPossible( supplierMulti.get().makeSingleValueDimensionVectorSelector(new NoFilterVectorOffset(10, 0, 1)), true, string -> string != null diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 653b39ff9bf..d223926101b 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -37,6 +37,7 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.filter.SelectorPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.AutoTypeColumnMerger; @@ -84,7 +85,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.SortedMap; import java.util.TreeMap; import java.util.TreeSet; @@ -529,9 +529,9 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest Assert.assertNull(dElementIndexSupplier.as(NullValueIndex.class)); - ImmutableBitmap sNullIndex = sNulls.get().computeBitmapResult(resultFactory); - ImmutableBitmap lNullIndex = lNulls.get().computeBitmapResult(resultFactory); - ImmutableBitmap dNullIndex = dNulls.get().computeBitmapResult(resultFactory); + ImmutableBitmap sNullIndex = sNulls.get().computeBitmapResult(resultFactory, false); + ImmutableBitmap lNullIndex = lNulls.get().computeBitmapResult(resultFactory, false); + ImmutableBitmap dNullIndex = dNulls.get().computeBitmapResult(resultFactory, false); int rowCounter = 0; while (offset.withinBounds()) { @@ -707,26 +707,26 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest Assert.assertEquals(theString, dimSelectorLookupVal); Assert.assertEquals(dimSelector.idLookup().lookupId(dimSelectorLookupVal), dimSelector.getRow().get(0)); - Assert.assertTrue(valueSetIndex.forValue(theString).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(valueSetIndex.forValue(theString).computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(theString))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(theString)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(rowNumber)); - Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); - Assert.assertTrue(dimSelector.makeValueMatcher(x -> Objects.equals(x, theString)).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches(false)); + Assert.assertTrue(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(theString)).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(NO_MATCH)).matches(false)); } else { Assert.assertNull(valueSelector.getObject()); Assert.assertTrue(path, valueSelector.isNull()); @@ -735,23 +735,23 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest Assert.assertNull(dimSelector.getObject()); Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0))); - Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(rowNumber)); + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(rowNumber)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); - Assert.assertTrue(dimSelector.makeValueMatcher(x -> x == null).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches(false)); + Assert.assertTrue(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(null)).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(NO_MATCH)).matches(false)); } } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index 2c2a69faf44..7b3a8eac20b 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -36,6 +36,7 @@ import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.filter.SelectorPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.DimensionSelector; @@ -80,7 +81,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.SortedMap; import java.util.TreeMap; import java.util.TreeSet; @@ -312,7 +312,7 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest Assert.assertNotNull(valueSetIndex); BitmapColumnIndex indexForValue = valueSetIndex.forValue(firstValue); Assert.assertEquals(firstValue, selector.getObject()); - Assert.assertTrue(indexForValue.computeBitmapResult(resultFactory).get(0)); + Assert.assertTrue(indexForValue.computeBitmapResult(resultFactory, false).get(0)); } } @Test @@ -349,7 +349,7 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest Assert.assertNotNull(valueSetIndex); BitmapColumnIndex indexForValue = valueSetIndex.forValue(firstValue); Assert.assertEquals(firstValue, selector.getObject()); - Assert.assertTrue(indexForValue.computeBitmapResult(resultFactory).get(0)); + Assert.assertTrue(indexForValue.computeBitmapResult(resultFactory, false).get(0)); } } private void smokeTest(NestedDataComplexColumn column) throws IOException @@ -465,45 +465,45 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest String dimSelectorLookupVal = dimSelector.lookupName(dimSelector.getRow().get(0)); Assert.assertEquals(theString, dimSelectorLookupVal); Assert.assertEquals(dimSelector.idLookup().lookupId(dimSelectorLookupVal), dimSelector.getRow().get(0)); - Assert.assertTrue(valueSetIndex.forValue(theString).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(valueSetIndex.forValue(theString).computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(theString))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(theString)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); - Assert.assertTrue(dimSelector.makeValueMatcher(x -> Objects.equals(x, theString)).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(rowNumber)); + Assert.assertTrue(dimSelector.makeValueMatcher(theString).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches(false)); + Assert.assertTrue(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(theString)).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(NO_MATCH)).matches(false)); } else { Assert.assertNull(valueSelector.getObject()); Assert.assertTrue(path, valueSelector.isNull()); Assert.assertEquals(0, dimSelector.getRow().get(0)); Assert.assertNull(dimSelector.getObject()); Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0))); - Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory, false).get(rowNumber)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(rowNumber)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(rowNumber)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(rowNumber)); - Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); - Assert.assertTrue(dimSelector.makeValueMatcher(x -> x == null).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches(false)); + Assert.assertTrue(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(null)).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(NO_MATCH)).matches(false)); } } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java index a691f4470a9..72f7dfafdff 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java @@ -171,7 +171,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); Assert.assertEquals(0, bitmap.size()); } @@ -190,21 +190,21 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("b"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 7, 8); // non-existent in local column columnIndex = valueSetIndex.forValue("fo"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); // set index columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("b", "fooo", "z"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.8, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 2, 3, 4, 5, 6, 7, 8); } @@ -224,144 +224,144 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex forRange = rangeIndex.forRange(null, false, "a", false); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange(null, true, "a", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange(null, false, "b", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange(null, false, "b", false); Assert.assertNotNull(forRange); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 7, 8); forRange = rangeIndex.forRange("a", false, "b", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange("a", true, "b", false); Assert.assertNotNull(forRange); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 7, 8); forRange = rangeIndex.forRange("b", false, "fon", false); Assert.assertNotNull(forRange); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 7, 8); forRange = rangeIndex.forRange("bb", false, "fon", false); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange("b", true, "foo", false); Assert.assertNotNull(forRange); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 9); forRange = rangeIndex.forRange("f", true, "g", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 5, 9); forRange = rangeIndex.forRange(null, false, "g", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 5, 7, 8, 9); forRange = rangeIndex.forRange("f", false, null, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 5, 6, 9); forRange = rangeIndex.forRange("b", true, "fooo", true); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 9); forRange = rangeIndex.forRange("b", true, "fooo", false); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 5, 9); forRange = rangeIndex.forRange(null, true, "fooo", true); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 3, 7, 8, 9); forRange = rangeIndex.forRange("b", true, null, false); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 5, 6, 9); forRange = rangeIndex.forRange("b", false, null, true); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); forRange = rangeIndex.forRange(null, true, "fooo", false); Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 5, 7, 8, 9); forRange = rangeIndex.forRange(null, true, null, true); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); forRange = rangeIndex.forRange(null, false, null, false); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); forRange = rangeIndex.forRange(null, true, "foa", false); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 7, 8); forRange = rangeIndex.forRange(null, true, "foooa", false); Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 5, 7, 8, 9); forRange = rangeIndex.forRange("foooa", true, "ggg", false); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange("g", true, "gg", false); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange("z", true, "zz", false); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange("z", false, "zz", false); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 4, 6); } @@ -385,7 +385,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT s -> !"fooo".equals(s) ); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 9); forRange = rangeIndex.forRange( @@ -396,7 +396,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT s -> "fooo".equals(s) ); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 5); forRange = rangeIndex.forRange( @@ -407,7 +407,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT s -> !"fooo".equals(s) ); Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 8, 9); forRange = rangeIndex.forRange( @@ -418,7 +418,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT s -> !"fooo".equals(s) ); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 3, 7, 8, 9); forRange = rangeIndex.forRange( @@ -429,7 +429,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT s -> true ); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 5, 6, 9); } @@ -452,7 +452,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 4, 6, 7, 8); } @@ -471,7 +471,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 7, 8); } @@ -490,21 +490,21 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("b"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 3); // non-existent in local column columnIndex = valueSetIndex.forValue("fo"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); // set index columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("b", "fooo", "z"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 3, 4, 5, 6); } @@ -524,77 +524,77 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT Assert.assertNotNull(forRange); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 5, 9); forRange = rangeIndex.forRange(null, false, "g", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 3, 5, 9); forRange = rangeIndex.forRange(null, false, "a", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange(null, false, "b", true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndex.forRange(null, false, "b", false); Assert.assertNotNull(forRange); Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 3); forRange = rangeIndex.forRange("f", false, null, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 5, 6, 9); forRange = rangeIndex.forRange("b", true, "fooo", true); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 9); forRange = rangeIndex.forRange("b", true, "fooo", false); Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 5, 9); forRange = rangeIndex.forRange(null, true, "fooo", true); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 3, 9); forRange = rangeIndex.forRange("b", true, null, false); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 5, 6, 9); forRange = rangeIndex.forRange("b", false, null, true); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 3, 4, 5, 6, 9); forRange = rangeIndex.forRange(null, true, "fooo", false); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 3, 5, 9); forRange = rangeIndex.forRange(null, true, null, true); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 3, 4, 5, 6, 9); forRange = rangeIndex.forRange(null, false, null, false); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 3, 4, 5, 6, 9); } @@ -617,7 +617,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 3, 4, 6); } @@ -639,14 +639,14 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("1"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 9); // set index columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1", "300", "700"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 2, 3, 7, 8, 9); } @@ -666,56 +666,56 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT Assert.assertNotNull(forRange); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 6, 7, 8); forRange = rangeIndexes.forRange(1, true, 3, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(1, false, 3, true); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 9); forRange = rangeIndexes.forRange(1, false, 3, false); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 4, 5, 9); forRange = rangeIndexes.forRange(100L, true, 300L, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(100L, true, 300L, false); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 7, 8); forRange = rangeIndexes.forRange(100L, false, 300L, true); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 6); forRange = rangeIndexes.forRange(100L, false, 300L, false); Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 6, 7, 8); forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); } @@ -738,7 +738,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 4, 5, 9); } @@ -757,7 +757,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 5, 8); } @@ -776,14 +776,14 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("3"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 4); // set index columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1", "3", "300"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 4, 7, 9); // set index with null @@ -795,14 +795,14 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT columnIndex = valueSetIndex.forSortedValues(treeSet); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.8, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 2, 3, 4, 5, 7, 8, 9); // null value should really use NullValueIndex, but this works for classic reasons columnIndex = valueSetIndex.forValue(null); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 5, 8); } @@ -822,52 +822,52 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT Assert.assertNotNull(forRange); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 6, 7); forRange = rangeIndexes.forRange(100, true, 300, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(100, false, 300, true); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 6); forRange = rangeIndexes.forRange(100, true, 300, false); Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 7); forRange = rangeIndexes.forRange(100, false, 300, false); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 6, 7); forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 9); forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 9); forRange = rangeIndexes.forRange(null, false, 0, false); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(null, false, 1, false); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 9); forRange = rangeIndexes.forRange(null, false, 1, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); } @@ -890,7 +890,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 4, 6); } @@ -912,14 +912,14 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("1.2"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 4, 7); // set index columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1.2", "3.3", "6.6"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.7, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 3, 4, 5, 6, 7, 9); } @@ -939,80 +939,80 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT Assert.assertNotNull(forRange); Assert.assertEquals(0.9, forRange.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 6, 7, 8, 9); forRange = rangeIndexes.forRange(1.1, false, 3.3, false); Assert.assertNotNull(forRange); Assert.assertEquals(0.9, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 6, 7, 8, 9); forRange = rangeIndexes.forRange(1.1, true, 3.3, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 4, 7); forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9); forRange = rangeIndexes.forRange(1.111, true, 1.19, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(1.01, true, 1.09, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(0.05, true, 0.98, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(0.05, true, 1.1, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(8.99, true, 10.10, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(8.99, true, 10.10, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(10.00, true, 10.10, true); Assert.assertNotNull(forRange); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); } @@ -1035,7 +1035,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 3, 4, 6, 7, 9); } @@ -1054,7 +1054,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 6); } @@ -1073,14 +1073,14 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("6.6"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 5); // set index columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1.2", "3.3", "7.7"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 4, 7, 9); // set index with null @@ -1092,14 +1092,14 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT columnIndex = valueSetIndex.forSortedValues(treeSet); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.7, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 2, 3, 4, 6, 7, 9); // null value should really use NullValueIndex, but this works for classic reasons columnIndex = valueSetIndex.forValue(null); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 6); } @@ -1119,37 +1119,37 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT Assert.assertNotNull(forRange); Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 7, 8, 9); forRange = rangeIndexes.forRange(null, true, null, true); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 5, 7, 8, 9); forRange = rangeIndexes.forRange(null, false, null, false); Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 2, 4, 5, 7, 8, 9); forRange = rangeIndexes.forRange(null, true, 1.0, true); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); forRange = rangeIndexes.forRange(null, true, 1.1, false); Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 8); forRange = rangeIndexes.forRange(6.6, false, null, false); Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 5); forRange = rangeIndexes.forRange(6.6, true, null, false); Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = forRange.computeBitmapResult(bitmapResultFactory); + bitmap = forRange.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); } @@ -1172,7 +1172,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 4, 7, 9); } @@ -1194,7 +1194,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = nullIndex.get(); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 7); } @@ -1213,26 +1213,26 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("b"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 9); columnIndex = valueSetIndex.forValue("1"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 5); columnIndex = valueSetIndex.forValue("1.1"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 8); // set index columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("b", "300", "9.9", "1.6"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 4, 9); // set index with null @@ -1245,14 +1245,14 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT columnIndex = valueSetIndex.forSortedValues(treeSet); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 2, 3, 4, 7, 9); // null value should really use NullValueIndex, but this works for classic reasons columnIndex = valueSetIndex.forValue(null); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 7); } @@ -1287,7 +1287,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 1, 3, 4, 6, 9); } @@ -1443,19 +1443,19 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT BitmapColumnIndex columnIndex = valueSetIndex.forValue("1"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3333, columnIndex.estimateSelectivity(3), 0.001); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0); columnIndex = valueSetIndex.forValue("-2"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.3333, columnIndex.estimateSelectivity(3), 0.001); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2); columnIndex = valueSetIndex.forValue("2"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(3), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java index ac7bd1c6849..cebb40e7756 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -280,32 +280,32 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest Assert.assertNull(nullVector); } - Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(valueIndexes.forValue(row, ColumnType.DOUBLE).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertTrue(valueIndexes.forValue(row, ColumnType.DOUBLE).computeBitmapResult(resultFactory, false).get(i)); Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(String.valueOf(row)))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(String.valueOf(row))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); } else { if (NullHandling.sqlCompatible()) { Assert.assertNull(valueSelector.getObject()); Assert.assertTrue(valueSelector.isNull()); Assert.assertTrue(vectorValueSelector.getNullVector()[0]); - Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); } else { Assert.assertEquals(NullHandling.defaultDoubleValue(), valueSelector.getObject()); @@ -313,24 +313,24 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest Assert.assertEquals(NullHandling.defaultDoubleValue(), vectorValueSelector.getDoubleVector()[0], 0.0); Assert.assertNull(vectorValueSelector.getNullVector()); - Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); final String defaultString = String.valueOf(NullHandling.defaultDoubleValue()); - Assert.assertTrue(valueSetIndex.forValue(defaultString).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forValue(defaultString).computeBitmapResult(resultFactory, false).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(defaultString)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); } - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java index ebd27fa794d..945bb1e9e9e 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -280,53 +280,53 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest Assert.assertNull(nullVector); } - Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(valueIndexes.forValue(row, ColumnType.LONG).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forValue(String.valueOf(row)).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertTrue(valueIndexes.forValue(row, ColumnType.LONG).computeBitmapResult(resultFactory, false).get(i)); Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(String.valueOf(row)))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(String.valueOf(row))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); } else { if (NullHandling.sqlCompatible()) { Assert.assertNull(valueSelector.getObject()); Assert.assertTrue(valueSelector.isNull()); Assert.assertTrue(vectorValueSelector.getNullVector()[0]); - Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); } else { Assert.assertEquals(NullHandling.defaultLongValue(), valueSelector.getObject()); Assert.assertFalse(valueSelector.isNull()); Assert.assertNull(vectorValueSelector.getNullVector()); - Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(null).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); final String defaultString = String.valueOf(NullHandling.defaultLongValue()); - Assert.assertTrue(valueSetIndex.forValue(defaultString).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forValue(defaultString).computeBitmapResult(resultFactory, false).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(defaultString)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); } - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index 8b48534ac56..3f59521be73 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -33,7 +33,9 @@ import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.java.util.common.io.smoosh.SmooshedWriter; import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.query.filter.SelectorPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.segment.AutoTypeColumnIndexer; import org.apache.druid.segment.AutoTypeColumnMerger; import org.apache.druid.segment.ColumnValueSelector; @@ -69,10 +71,8 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Objects; import java.util.SortedMap; import java.util.TreeMap; -import java.util.TreeSet; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; @@ -271,27 +271,27 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest Assert.assertEquals(row, dimSelectorLookupVal); Assert.assertEquals(dimSelector.idLookup().lookupId(dimSelectorLookupVal), dimSelector.getRow().get(0)); - Assert.assertTrue(valueSetIndex.forValue(row).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(valueIndexes.forValue(row, ColumnType.STRING).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(row))) - .computeBitmapResult(resultFactory) + Assert.assertTrue(valueSetIndex.forValue(row).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertTrue(valueIndexes.forValue(row, ColumnType.STRING).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertTrue(valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of(row))) + .computeBitmapResult(resultFactory, false) .get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(row)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of(NO_MATCH))) - .computeBitmapResult(resultFactory) + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertFalse(valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of(NO_MATCH))) + .computeBitmapResult(resultFactory, false) .get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); - Assert.assertTrue(dimSelector.makeValueMatcher(row).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); - Assert.assertTrue(dimSelector.makeValueMatcher(x -> Objects.equals(x, row)).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher(row).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches(false)); + Assert.assertTrue(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(row)).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(NO_MATCH)).matches(false)); } else { Assert.assertNull(valueSelector.getObject()); @@ -299,21 +299,21 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest Assert.assertNull(dimSelector.getObject()); Assert.assertNull(dimSelector.lookupName(dimSelector.getRow().get(0))); - Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory).get(i)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); - Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueSetIndex.forValue(null).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); Assert.assertTrue(predicateIndex.forPredicate(new SelectorPredicateFactory(null)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(valueSetIndex.forValue(NO_MATCH).computeBitmapResult(resultFactory, false).get(i)); Assert.assertFalse(predicateIndex.forPredicate(new SelectorPredicateFactory(NO_MATCH)) - .computeBitmapResult(resultFactory) + .computeBitmapResult(resultFactory, false) .get(i)); - Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches()); - Assert.assertTrue(dimSelector.makeValueMatcher(x -> x == null).matches()); - Assert.assertFalse(dimSelector.makeValueMatcher(x -> Objects.equals(x, NO_MATCH)).matches()); + Assert.assertTrue(dimSelector.makeValueMatcher((String) null).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(NO_MATCH).matches(false)); + Assert.assertTrue(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(null)).matches(false)); + Assert.assertFalse(dimSelector.makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(NO_MATCH)).matches(false)); } offset.increment(); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 401e5d87ebd..e394668ed80 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -408,9 +408,13 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest Assert.assertArrayEquals(((List) row).toArray(), (Object[]) valueSelector.getObject()); if (expectedType.getSingleType() != null) { Assert.assertArrayEquals(((List) row).toArray(), (Object[]) vectorObjectSelector.getObjectVector()[0]); - Assert.assertTrue(valueIndexes.forValue(row, expectedType.getSingleType()).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(valueIndexes.forValue(row, expectedType.getSingleType()).computeBitmapResult(resultFactory, + false + ).get(i)); for (Object o : ((List) row)) { - Assert.assertTrue("Failed on row: " + row, arrayElementIndexes.containsValue(o, expectedType.getSingleType().getElementType()).computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue("Failed on row: " + row, arrayElementIndexes.containsValue(o, expectedType.getSingleType().getElementType()).computeBitmapResult(resultFactory, + false + ).get(i)); } } else { // mixed type vector object selector coerces to the most common type @@ -442,7 +446,7 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest } } } - Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); } else { Assert.assertNull(valueSelector.getObject()); @@ -454,9 +458,11 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest Assert.assertNull(dimensionVectorSelector.lookupName(dimensionVectorSelector.getRowVector()[0])); } } - Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory).get(i)); + Assert.assertTrue(nullValueIndex.get().computeBitmapResult(resultFactory, false).get(i)); if (expectedType.getSingleType() != null) { - Assert.assertFalse(arrayElementIndexes.containsValue(null, expectedType.getSingleType()).computeBitmapResult(resultFactory).get(i)); + Assert.assertFalse(arrayElementIndexes.containsValue(null, expectedType.getSingleType()).computeBitmapResult(resultFactory, + false + ).get(i)); } } diff --git a/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java index 8843bd78856..ebf39d5c0d6 100644 --- a/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/serde/DictionaryEncodedStringIndexSupplierTest.java @@ -24,6 +24,7 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.bitmap.MutableBitmap; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.DefaultBitmapResultFactory; +import org.apache.druid.query.filter.InDimFilter; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.GenericIndexedWriter; @@ -40,7 +41,6 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; import java.nio.charset.StandardCharsets; -import java.util.TreeSet; public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHandlingTest { @@ -63,42 +63,42 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan BitmapColumnIndex columnIndex = valueSetIndex.forValue("b"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.1, columnIndex.estimateSelectivity(10), 0.0); - ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 3); // non-existent in local column columnIndex = valueSetIndex.forValue("fo"); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(10), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); // set index - columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("b", "fooo", "z"))); + columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("b", "fooo", "z"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.5, columnIndex.estimateSelectivity(10), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 2, 3, 4, 5, 6); // set index with single value in middle - columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("foo"))); + columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("foo"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.2, columnIndex.estimateSelectivity(10), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap, 0, 9); // set index with no element in column and all elements less than lowest non-null value - columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("a", "aa", "aaa"))); + columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("a", "aa", "aaa"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(10), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); // set index with no element in column and all elements greater than highest non-null value - columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("zz", "zzz", "zzzz"))); + columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("zz", "zzz", "zzzz"))); Assert.assertNotNull(columnIndex); Assert.assertEquals(0.0, columnIndex.estimateSelectivity(10), 0.0); - bitmap = columnIndex.computeBitmapResult(bitmapResultFactory); + bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false); checkBitmap(bitmap); } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java index cacec12e666..27f5113eea2 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/DummyStringVirtualColumn.java @@ -19,10 +19,10 @@ package org.apache.druid.segment.virtual; -import com.google.common.base.Predicate; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.dimension.DefaultDimensionSpec; import org.apache.druid.query.dimension.DimensionSpec; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnSelector; @@ -248,7 +248,7 @@ public class DummyStringVirtualColumn implements VirtualColumn } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { throw new UnsupportedOperationException("not supported"); } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java index 89ddc9bfe30..c8fd92a1079 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/ExpressionVirtualColumnTest.java @@ -19,7 +19,6 @@ package org.apache.druid.segment.virtual; -import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -35,6 +34,8 @@ import org.apache.druid.query.dimension.DimensionSpec; import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.BucketExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; +import org.apache.druid.query.filter.StringPredicateDruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.BaseFloatColumnValueSelector; @@ -294,9 +295,9 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest } @Override - public ValueMatcher makeValueMatcher(Predicate predicate) + public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory) { - return delegate.makeValueMatcher(predicate); + return delegate.makeValueMatcher(predicateFactory); } @Override @@ -472,38 +473,38 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest final ValueMatcher nullMatcher = selector.makeValueMatcher((String) null); final ValueMatcher fiveMatcher = selector.makeValueMatcher("5"); - final ValueMatcher nonNullMatcher = selector.makeValueMatcher(Predicates.notNull()); + final ValueMatcher nonNullMatcher = selector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.notNull(), false)); CURRENT_ROW.set(ROW0); - Assert.assertEquals(true, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(false, nonNullMatcher.matches()); + Assert.assertEquals(true, nullMatcher.matches(false)); + Assert.assertEquals(false, fiveMatcher.matches(false)); + Assert.assertEquals(false, nonNullMatcher.matches(false)); Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW1); if (NullHandling.replaceWithDefault()) { - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals(false, nullMatcher.matches(false)); + Assert.assertEquals(false, fiveMatcher.matches(false)); + Assert.assertEquals(true, nonNullMatcher.matches(false)); Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0))); } else { // y is null in row1 - Assert.assertEquals(true, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(false, nonNullMatcher.matches()); + Assert.assertEquals(true, nullMatcher.matches(false)); + Assert.assertEquals(false, fiveMatcher.matches(false)); + Assert.assertEquals(false, nonNullMatcher.matches(false)); Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); } CURRENT_ROW.set(ROW2); - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals(false, nullMatcher.matches(false)); + Assert.assertEquals(false, fiveMatcher.matches(false)); + Assert.assertEquals(true, nonNullMatcher.matches(false)); Assert.assertEquals("5.1", selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW3); - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(true, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals(false, nullMatcher.matches(false)); + Assert.assertEquals(true, fiveMatcher.matches(false)); + Assert.assertEquals(true, nonNullMatcher.matches(false)); Assert.assertEquals("5", selector.lookupName(selector.getRow().get(0))); } @@ -515,10 +516,10 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest COLUMN_SELECTOR_FACTORY ); - final ValueMatcher nonNullMatcher = selector.makeValueMatcher(Predicates.notNull()); + final ValueMatcher nonNullMatcher = selector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.notNull(), false)); CURRENT_ROW.set(ROW0); - Assert.assertEquals(false, nonNullMatcher.matches()); + Assert.assertEquals(false, nonNullMatcher.matches(false)); } @@ -596,38 +597,38 @@ public class ExpressionVirtualColumnTest extends InitializedNullHandlingTest final ValueMatcher nullMatcher = selector.makeValueMatcher((String) null); final ValueMatcher fiveMatcher = selector.makeValueMatcher("5"); - final ValueMatcher nonNullMatcher = selector.makeValueMatcher(Predicates.notNull()); + final ValueMatcher nonNullMatcher = selector.makeValueMatcher(StringPredicateDruidPredicateFactory.of(Predicates.notNull(), false)); CURRENT_ROW.set(ROW0); - Assert.assertEquals(true, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(false, nonNullMatcher.matches()); + Assert.assertEquals(true, nullMatcher.matches(false)); + Assert.assertEquals(false, fiveMatcher.matches(false)); + Assert.assertEquals(false, nonNullMatcher.matches(false)); Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW1); if (NullHandling.replaceWithDefault()) { - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals(false, nullMatcher.matches(false)); + Assert.assertEquals(false, fiveMatcher.matches(false)); + Assert.assertEquals(true, nonNullMatcher.matches(false)); Assert.assertEquals("4", selector.lookupName(selector.getRow().get(0))); } else { // y is null in row1 - Assert.assertEquals(true, nullMatcher.matches()); - Assert.assertEquals(false, fiveMatcher.matches()); - Assert.assertEquals(false, nonNullMatcher.matches()); + Assert.assertEquals(true, nullMatcher.matches(false)); + Assert.assertEquals(false, fiveMatcher.matches(false)); + Assert.assertEquals(false, nonNullMatcher.matches(false)); Assert.assertEquals(null, selector.lookupName(selector.getRow().get(0))); } CURRENT_ROW.set(ROW2); - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(true, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals(false, nullMatcher.matches(false)); + Assert.assertEquals(true, fiveMatcher.matches(false)); + Assert.assertEquals(true, nonNullMatcher.matches(false)); Assert.assertEquals("5.1", selector.lookupName(selector.getRow().get(0))); CURRENT_ROW.set(ROW3); - Assert.assertEquals(false, nullMatcher.matches()); - Assert.assertEquals(true, fiveMatcher.matches()); - Assert.assertEquals(true, nonNullMatcher.matches()); + Assert.assertEquals(false, nullMatcher.matches(false)); + Assert.assertEquals(true, fiveMatcher.matches(false)); + Assert.assertEquals(true, nonNullMatcher.matches(false)); Assert.assertEquals("5", selector.lookupName(selector.getRow().get(0))); } diff --git a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java index 737e86d7414..fbf3b1c3aee 100644 --- a/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/segment/virtual/VirtualColumnsTest.java @@ -20,7 +20,6 @@ package org.apache.druid.segment.virtual; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicate; import com.google.common.collect.ImmutableList; import com.google.common.primitives.Longs; import org.apache.druid.query.dimension.DefaultDimensionSpec; @@ -29,6 +28,7 @@ import org.apache.druid.query.dimension.ExtractionDimensionSpec; import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.extraction.BucketExtractionFn; import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.BaseFloatColumnValueSelector; @@ -579,9 +579,9 @@ public class VirtualColumnsTest extends InitializedNullHandlingTest } @Override - public ValueMatcher makeValueMatcher(final Predicate predicate) + public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory) { - return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicate); + return DimensionSelectorUtils.makeValueMatcherGeneric(this, predicateFactory); } @Override 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 1afbe4fa03a..086633d7e59 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 @@ -5753,9 +5753,7 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest new Object[]{"b"}, new Object[]{"b"}, new Object[]{"c"}, - new Object[]{""}, - new Object[]{null}, - new Object[]{null} + new Object[]{""} ) ); } 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 e8a72833960..908ccae687b 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 @@ -735,10 +735,15 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .setContext(queryContext) .build() ), - ImmutableList.of( + NullHandling.replaceWithDefault() + ? ImmutableList.of( new Object[]{NULL_STRING, 3L}, new Object[]{"xabc", 1L} ) + : ImmutableList.of( + new Object[]{null, 5L}, + new Object[]{"xabc", 1L} + ) ); } @@ -775,10 +780,15 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .setContext(queryContext) .build() ), - ImmutableList.of( + NullHandling.replaceWithDefault() + ? ImmutableList.of( new Object[]{NULL_STRING, 3L}, new Object[]{"xabc", 1L} ) + : + ImmutableList.of( + new Object[]{"xabc", 1L} + ) ); } @@ -823,10 +833,14 @@ public class CalciteJoinQueryTest extends BaseCalciteQueryTest .setContext(queryContext) .build() ), - ImmutableList.of( + NullHandling.replaceWithDefault() + ? ImmutableList.of( new Object[]{NULL_STRING, 6L}, new Object[]{"xabc", 2L} ) + : ImmutableList.of( + new Object[]{"xabc", 2L} + ) ); } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java index a1438824b40..c97271e8c39 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteParameterQueryTest.java @@ -323,11 +323,16 @@ public class CalciteParameterQueryTest extends BaseCalciteQueryTest .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of( + NullHandling.replaceWithDefault() + ? ImmutableList.of( new Object[]{"10.1", 1L}, new Object[]{"2", 1L}, new Object[]{"abc", 1L}, new Object[]{"def", 1L} + ) + : ImmutableList.of( + new Object[]{"2", 1L}, + new Object[]{"def", 1L} ), ImmutableList.of( new SqlParameter(SqlType.VARCHAR, "a"), diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 8659c1c29ef..08fdbc3bfaa 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -2676,12 +2676,17 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - ImmutableList.of( + NullHandling.replaceWithDefault() + ? ImmutableList.of( new Object[]{"10.1", 1L}, new Object[]{"2", 1L}, new Object[]{"abc", 1L}, new Object[]{"def", 1L} ) + : ImmutableList.of( + new Object[]{"2", 1L}, + new Object[]{"def", 1L} + ) ); } @@ -3104,12 +3109,12 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .context(QUERY_CONTEXT_DEFAULT) .build() ), + ImmutableList.of( - NullHandling.replaceWithDefault() ? + NullHandling.replaceWithDefault() // Matches everything but "abc" - new Object[]{5L} : - // match only null values - new Object[]{4L} + ? new Object[]{5L} + : new Object[]{2L} ) ); } @@ -5922,7 +5927,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .build() ), ImmutableList.of( - new Object[]{5L} + new Object[]{NullHandling.replaceWithDefault() ? 5L : 4L} ) ); } @@ -5975,7 +5980,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .build() ), ImmutableList.of( - new Object[]{2L} + new Object[]{NullHandling.replaceWithDefault() ? 2L : 1L} ) ); } @@ -6010,7 +6015,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .build() ), ImmutableList.of( - new Object[]{5L} + NullHandling.replaceWithDefault() ? new Object[]{5L} : new Object[]{4L} ) ); } @@ -7149,12 +7154,8 @@ public class CalciteQueryTest extends BaseCalciteQueryTest .setContext(QUERY_CONTEXT_DEFAULT) .build() ), - NullHandling.replaceWithDefault() ? ImmutableList.of( new Object[]{3L, 2L} - ) : - ImmutableList.of( - new Object[]{5L, 3L} ) ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java index 5a72d01b423..093ca0278c5 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/expression/ExpressionTestHelper.java @@ -389,6 +389,6 @@ class ExpressionTestHelper ) ); - Assert.assertEquals("Result for: " + rexNode, expectedResult, matcher.matches()); + Assert.assertEquals("Result for: " + rexNode, expectedResult, matcher.matches(false)); } }