You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by la...@apache.org on 2023/10/15 19:23:44 UTC
[druid] branch 28.0.0 updated: sql compatible three-valued logic native filters (#15058) (#15151)
This is an automated email from the ASF dual-hosted git repository.
lakshsingla pushed a commit to branch 28.0.0
in repository https://gitbox.apache.org/repos/asf/druid.git
The following commit(s) were added to refs/heads/28.0.0 by this push:
new 95d1e1a2784 sql compatible three-valued logic native filters (#15058) (#15151)
95d1e1a2784 is described below
commit 95d1e1a27841dab91a99b625145097455171e5fa
Author: Clint Wylie <cw...@apache.org>
AuthorDate: Sun Oct 15 12:23:37 2023 -0700
sql compatible three-valued logic native filters (#15058) (#15151)
* 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 +-
.../MapTypeMapVirtualColumnDimensionSelector.java | 8 +-
...tringTypeMapVirtualColumnDimensionSelector.java | 15 +-
.../druid/segment/MapVirtualColumnGroupByTest.java | 140 +++++++-
.../sql/DoublesSketchSqlAggregatorTest.java | 2 +-
.../apache/druid/query/filter/BloomDimFilter.java | 8 +
.../druid/query/filter/BloomDimFilterTest.java | 32 +-
.../common/SortMergeJoinFrameProcessor.java | 6 +-
.../scan/ExternalColumnSelectorFactory.java | 6 +-
.../org/apache/druid/msq/exec/MSQSelectTest.java | 6 +-
.../apache/druid/common/config/NullHandling.java | 18 +-
.../common/config/NullValueHandlingConfig.java | 39 ++-
.../druid/frame/field/StringFieldReader.java | 6 +-
.../processor/MultiColumnSelectorFactory.java | 6 +-
.../read/columnar/StringFrameColumnReader.java | 6 +-
.../druid/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 | 80 +++--
.../PredicateFilteredDimensionSelector.java | 21 +-
.../druid/query/filter/DruidPredicateFactory.java | 17 +
.../apache/druid/query/filter/EqualityFilter.java | 13 +-
.../java/org/apache/druid/query/filter/Filter.java | 8 +-
.../org/apache/druid/query/filter/InDimFilter.java | 8 +
.../apache/druid/query/filter/NotDimFilter.java | 5 +
.../org/apache/druid/query/filter/NullFilter.java | 7 +
.../org/apache/druid/query/filter/RangeFilter.java | 4 +-
.../query/filter/SelectorPredicateFactory.java | 8 +
.../StringPredicateDruidPredicateFactory.java | 99 ++++++
.../apache/druid/query/filter/ValueMatcher.java | 12 +-
.../filter/vector/ArrayVectorValueMatcher.java | 7 +-
.../filter/vector/BooleanVectorValueMatcher.java | 17 +-
.../filter/vector/DoubleVectorValueMatcher.java | 25 +-
.../filter/vector/FloatVectorValueMatcher.java | 25 +-
.../filter/vector/LongVectorValueMatcher.java | 24 +-
.../vector/MultiValueStringVectorValueMatcher.java | 36 +-
.../filter/vector/ObjectVectorValueMatcher.java | 16 +-
.../SingleValueStringVectorValueMatcher.java | 59 ++--
.../vector/StringObjectVectorValueMatcher.java | 13 +-
.../query/filter/vector/VectorValueMatcher.java | 284 ++++++++++++++-
.../filter/vector/VectorValueMatcherFactory.java | 34 --
.../epinephelinae/RowBasedGrouperHelper.java | 6 +-
.../query/groupby/having/DimFilterHavingSpec.java | 2 +-
.../rowsandcols/LazilyDecoratedRowsAndColumns.java | 2 +-
.../org/apache/druid/query/scan/ConcatCursor.java | 6 +-
.../druid/query/search/UseIndexesStrategy.java | 2 +-
.../segment/BaseSingleValueDimensionSelector.java | 17 +-
.../druid/segment/ConstantDimensionSelector.java | 10 +-
.../ConstantMultiValueDimensionSelector.java | 16 +-
.../segment/DictionaryEncodedColumnIndexer.java | 4 +-
.../segment/DictionaryEncodedColumnMerger.java | 4 +-
.../apache/druid/segment/DimensionSelector.java | 19 +-
.../druid/segment/DimensionSelectorUtils.java | 96 ++---
.../org/apache/druid/segment/FilteredOffset.java | 17 +-
.../org/apache/druid/segment/IndexMergerV9.java | 1 +
.../segment/RowBasedColumnSelectorFactory.java | 23 +-
.../org/apache/druid/segment/RowBasedCursor.java | 6 +-
.../segment/SingleScanTimeDimensionSelector.java | 17 +-
.../druid/segment/StringDimensionIndexer.java | 60 +++-
.../druid/segment/UnnestDimensionCursor.java | 19 +-
.../column/StringUtf8DictionaryEncodedColumn.java | 60 +++-
...ValueMatcher.java => AllFalseValueMatcher.java} | 15 +-
...eValueMatcher.java => AllTrueValueMatcher.java} | 13 +-
...lueMatcher.java => AllUnknownValueMatcher.java} | 27 +-
.../org/apache/druid/segment/filter/AndFilter.java | 18 +-
.../apache/druid/segment/filter/BoundFilter.java | 22 +-
.../segment/filter/ColumnComparisonFilter.java | 9 +-
.../druid/segment/filter/ConstantMatcherType.java | 81 +++++
.../segment/filter/DimensionPredicateFilter.java | 8 +
.../druid/segment/filter/ExpressionFilter.java | 47 ++-
.../apache/druid/segment/filter/FalseFilter.java | 7 +-
.../org/apache/druid/segment/filter/Filters.java | 11 +-
.../apache/druid/segment/filter/LikeFilter.java | 4 +-
.../org/apache/druid/segment/filter/NotFilter.java | 31 +-
.../org/apache/druid/segment/filter/OrFilter.java | 18 +-
.../filter/PredicateValueMatcherFactory.java | 34 +-
.../apache/druid/segment/filter/SpatialFilter.java | 11 +-
.../filter/StringConstantValueMatcherFactory.java | 1 +
.../apache/druid/segment/filter/TrueFilter.java | 5 +-
.../apache/druid/segment/filter/ValueMatchers.java | 328 +++++++++++++----
.../IncrementalIndexStorageAdapter.java | 10 +-
.../segment/index/AllFalseBitmapColumnIndex.java | 18 +-
.../segment/index/AllTrueBitmapColumnIndex.java | 2 +-
...Index.java => AllUnknownBitmapColumnIndex.java} | 24 +-
.../druid/segment/index/BitmapColumnIndex.java | 18 +-
.../index/IndexedStringDruidPredicateIndexes.java | 11 +
.../IndexedUtf8LexicographicalRangeIndexes.java | 27 +-
.../segment/index/IndexedUtf8ValueIndexes.java | 44 ++-
.../segment/index/SimpleImmutableBitmapIndex.java | 5 +-
.../index/SimpleImmutableBitmapIterableIndex.java | 20 +-
.../join/PossiblyNullDimensionSelector.java | 6 +-
.../apache/druid/segment/join/PostJoinCursor.java | 2 +-
.../join/table/IndexedTableDimensionSelector.java | 6 +-
.../nested/NestedFieldColumnIndexSupplier.java | 155 ++++++++-
.../nested/NestedFieldDictionaryEncodedColumn.java | 35 +-
.../nested/ScalarDoubleColumnAndIndexSupplier.java | 51 ++-
.../nested/ScalarLongColumnAndIndexSupplier.java | 51 ++-
.../apache/druid/segment/nested/VariantColumn.java | 38 +-
.../nested/VariantColumnAndIndexSupplier.java | 28 +-
.../settable/SettableDimensionValueSelector.java | 4 +-
.../druid/segment/transform/Transformer.java | 4 +-
.../druid/segment/vector/FilteredVectorOffset.java | 5 +-
.../ExpressionMultiValueDimensionSelector.java | 21 +-
.../druid/segment/virtual/ExpressionSelectors.java | 8 +-
.../segment/virtual/ListFilteredVirtualColumn.java | 210 +++++++----
.../segment/virtual/NestedFieldVirtualColumn.java | 6 +-
...erredEvaluationExpressionDimensionSelector.java | 6 +-
.../intset/ImmutableConciseSetTest.java | 20 ++
.../druid/frame/field/DoubleFieldReaderTest.java | 33 +-
.../druid/frame/field/FloatFieldReaderTest.java | 33 +-
.../druid/frame/field/LongFieldReaderTest.java | 33 +-
.../druid/frame/field/StringFieldReaderTest.java | 17 +-
.../query/aggregation/FilteredAggregatorTest.java | 6 +-
.../cardinality/CardinalityAggregatorTest.java | 6 +-
.../dimension/StringDimensionSelectorForTest.java | 6 +-
.../query/dimension/TestDimensionSelector.java | 6 +-
.../apache/druid/query/filter/InDimFilterTest.java | 6 +-
.../query/filter/SelectorPredicateFactoryTest.java | 2 +-
...ctorValueMatcherColumnProcessorFactoryTest.java | 15 +-
.../groupby/GroupByTimeseriesQueryRunnerTest.java | 29 ++
.../timeseries/TimeseriesQueryRunnerTest.java | 71 ++--
.../topn/TopNMetricSpecOptimizationsTest.java | 4 +-
.../ColumnSelectorColumnIndexSelectorTest.java | 5 +-
.../segment/ConstantDimensionSelectorTest.java | 22 +-
.../ConstantMultiValueDimensionSelectorTest.java | 49 +--
.../org/apache/druid/segment/IndexBuilder.java | 2 +-
.../druid/segment/IndexMergerNullHandlingTest.java | 3 +-
.../apache/druid/segment/IndexMergerTestBase.java | 2 +-
.../java/org/apache/druid/segment/ListCursor.java | 4 +-
.../druid/segment/filter/BaseFilterTest.java | 44 ++-
.../druid/segment/filter/BoundFilterTest.java | 11 +-
.../filter/DimensionPredicateFilterTest.java | 2 +-
.../druid/segment/filter/EqualityFilterTests.java | 387 ++++++++++++++++++++-
.../ExpressionFilterNonStrictBooleansTest.java | 90 +++++
.../druid/segment/filter/ExpressionFilterTest.java | 115 ++++--
.../segment/filter/FilterCnfConversionTest.java | 2 +-
.../druid/segment/filter/FilterPartitionTest.java | 8 +-
.../apache/druid/segment/filter/InFilterTest.java | 159 ++++++++-
.../druid/segment/filter/LikeFilterTest.java | 123 ++++++-
.../druid/segment/filter/NullFilterTest.java | 237 -------------
.../druid/segment/filter/NullFilterTests.java | 343 ++++++++++++++++++
.../filter/PredicateValueMatcherFactoryTest.java | 60 ++--
.../druid/segment/filter/RangeFilterTests.java | 170 ++++++++-
.../druid/segment/filter/ValueMatchersTest.java | 32 +-
.../nested/NestedDataColumnSupplierTest.java | 50 +--
.../nested/NestedDataColumnSupplierV4Test.java | 48 +--
.../nested/NestedFieldColumnIndexSupplierTest.java | 246 ++++++-------
.../nested/ScalarDoubleColumnSupplierTest.java | 38 +-
.../nested/ScalarLongColumnSupplierTest.java | 38 +-
.../nested/ScalarStringColumnSupplierTest.java | 52 +--
.../segment/nested/VariantColumnSupplierTest.java | 16 +-
.../DictionaryEncodedStringIndexSupplierTest.java | 22 +-
.../segment/virtual/DummyStringVirtualColumn.java | 4 +-
.../virtual/ExpressionVirtualColumnTest.java | 75 ++--
.../druid/segment/virtual/VirtualColumnsTest.java | 6 +-
.../druid/sql/calcite/CalciteArraysQueryTest.java | 4 +-
.../druid/sql/calcite/CalciteJoinQueryTest.java | 20 +-
.../sql/calcite/CalciteParameterQueryTest.java | 7 +-
.../apache/druid/sql/calcite/CalciteQueryTest.java | 25 +-
.../calcite/expression/ExpressionTestHelper.java | 2 +-
164 files changed, 4312 insertions(+), 1556 deletions(-)
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 det [...]
|`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 counte [...]
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<String> 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<String> predicate)
+ public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
{
+ final Predicate<String> 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<ResultRow> 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<ResultRow> result = runner.run(QueryPlus.wrap(query)).toList();
+ final List<ResultRow> 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<ResultRow> result = runner.run(QueryPlus.wrap(query)).toList();
+ final List<ResultRow> 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<ResultRow> result = runner.run(QueryPlus.wrap(query)).toList();
+ final List<ResultRow> 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<ResultRow> result = runner.run(QueryPlus.wrap(query)).toList();
+ final List<ResultRow> 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<String> 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<InputRow> 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<Object>
}
@Override
- public ValueMatcher makeValueMatcher(Predicate<String> 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<String> 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 030f352633f..30bbb850fbc 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
@@ -798,7 +798,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 8e6f242e9a3..365cff1a321 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.data.Indexed;
import javax.annotation.Nullable;
@@ -60,13 +61,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);
}
/**
@@ -98,6 +105,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<String> 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<String> 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<String> 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<String> predicate)
+ public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
{
+ final Predicate<String> 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<String> matcherPredicate)
+ public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
{
- final boolean matchNull = matcherPredicate.apply(null);
+ final Predicate<String> 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<String> 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<DruidLongPredicate> longPredicateSupplier;
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
private final Supplier<DruidDoublePredicate> 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<String> predicate, boolean isNullInputUnknown)
+ {
+ return new StringPredicateDruidPredicateFactory(predicate, isNullInputUnknown);
+ }
+
+ private final boolean isNullInputUnknown;
+ @Nullable
+ private final Predicate<String> predicate;
+
+ public StringPredicateDruidPredicateFactory(Predicate<String> predicate, boolean isNullInputUnknown)
+ {
+ this.predicate = predicate;
+ this.isNullInputUnknown = isNullInputUnknown;
+ }
+
+ @Override
+ public Predicate<String> 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<Object[]> 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<String> predicate)
+ private VectorValueMatcher makeMatcher(final Predicate<String> 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<String> 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<String> 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.
- *
+ * <p>
* 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<String> 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<String> predicate)
+ public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
{
+ final Predicate<String> 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<String> predicate)
+ public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
{
- return BooleanValueMatcher.of(predicate.apply(value));
+ final Predicate<String> 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<String> 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<String> predicate)
+ public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
{
- return BooleanValueMatcher.of(values.stream().anyMatch(predicate::apply));
+ final Predicate<String> 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<KeyType, ActualType extends
}
@Override
- public ValueMatcher makeValueMatcher(Predicate<String> 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<T extends Comparable<T>> imp
}
@Override
- public ValueMatcher makeValueMatcher(Predicate<String> 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<Object>, Dimensio
*/
ValueMatcher makeValueMatcher(@Nullable String value);
- ValueMatcher makeValueMatcher(Predicate<String> 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<Object>, 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<String> predicate)
+ public ValueMatcher makeValueMatcher(DruidPredicateFactory predicateFactory)
{
- return BooleanValueMatcher.of(predicate.apply(null));
+ final Predicate<String> 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<String> 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<String> 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<String> 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<String> predicate
+ final DruidPredicateFactory predicateFactory
)
{
- final boolean matchNull = predicate.apply(null);
+ final Predicate<String> 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<T> 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<T> implements ColumnSelectorFactory
}
@Override
- public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
+ public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
{
- final boolean matchNull = predicate.apply(null);
+ final Predicate<String> 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<T> 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<RowType> 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<RowType> 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<String> predicate)
+ public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
{
+ final Predicate<String> 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<int[]
{
if (extractionFn == null) {
final int valueId = lookupId(value);
+ final int nullValueId = lookupId(null);
if (valueId >= 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<int[]
}
};
} else {
- return BooleanValueMatcher.of(false);
+ return new ValueMatcher()
+ {
+ @Override
+ public boolean matches(boolean includeUnknown)
+ {
+ if (includeUnknown) {
+ IndexedInts row = getRow();
+ final int size = row.size();
+ if (size == 0) {
+ return true;
+ }
+ for (int i = 0; i < size; i++) {
+ if (row.get(i) == nullValueId) {
+ return true;
+ }
+ }
+ }
+ return false;
+ }
+
+ @Override
+ public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+ {
+ // nothing to inspect
+ }
+ };
}
} else {
// Employ caching BitSet optimization
- return makeValueMatcher(Predicates.equalTo(value));
+ return makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(value));
}
}
@Override
- public ValueMatcher makeValueMatcher(final Predicate<String> 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<String> 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<String> 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<String> 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<String> predicate)
+ public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
{
final BitSet checkedIds = new BitSet(getCardinality());
final BitSet matchingIds = new BitSet(getCardinality());
+ final Predicate<String> 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<String> row = new ArrayList<>(ithRow.size());
- // noinspection SSBasedInspection
- for (int j = 0; j < ithRow.size(); j++) {
+ List<String> 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 INSTANCE;
+ }
+
+ private AllUnknownValueMatcher()
+ {
+ }
+
+ @Override
+ public boolean matches(boolean includeUnknown)
{
- return matches ? TrueValueMatcher.instance() : FalseValueMatcher.instance();
+ return includeUnknown;
}
- private BooleanValueMatcher()
+ @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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
final List<T> 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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<DruidLongPredicate> longPredicateSupplier;
private final Supplier<DruidFloatPredicate> floatPredicateSupplier;
private final Supplier<DruidDoublePredicate> 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<DimensionSpec> dimensions;
@@ -79,13 +84,13 @@ public class ColumnComparisonFilter implements Filter
public static ValueMatcher makeValueMatcher(final List<Supplier<String[]>> 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<String> 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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<Valu
{
if (selector instanceof NilColumnValueSelector) {
// Column does not exist, or is unfilterable. Treat it as all nulls.
- return BooleanValueMatcher.of(predicateFactory.makeArrayPredicate(columnCapabilities).apply(null));
+
+ final boolean matchesNull = predicateFactory.makeArrayPredicate(columnCapabilities).apply(null);
+ if (matchesNull) {
+ return ValueMatchers.allTrue();
+ }
+ return ValueMatchers.makeAlwaysFalseObjectMatcher(selector);
} else {
// use the object predicate
final Predicate<Object[]> 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<Valu
{
if (selector instanceof NilColumnValueSelector) {
// Column does not exist, or is unfilterable. Treat it as all nulls.
- return BooleanValueMatcher.of(predicateFactory.makeStringPredicate().apply(null));
+ final boolean predicateMatches = predicateFactory.makeStringPredicate().apply(null);
+ if (predicateMatches) {
+ return ValueMatchers.allTrue();
+ }
+ return ValueMatchers.makeAlwaysFalseObjectMatcher(selector);
} else if (!isNumberOrString(selector.classOfObject())) {
// if column is definitely not a number of string, use the object predicate
final Predicate<Object> 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<Valu
private Predicate<Object[]> 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<Valu
if (rowValueStrings.isEmpty()) {
// Empty list is equivalent to null.
- return getStringPredicate().apply(null);
+ return matchNull || getStringPredicate().apply(null);
}
for (String rowValueString : rowValueStrings) {
- if (getStringPredicate().apply(NullHandling.emptyToNullIfNeeded(rowValueString))) {
+ final String coerced = NullHandling.emptyToNullIfNeeded(rowValueString);
+ if ((matchNull && coerced == null) || getStringPredicate().apply(coerced)) {
return true;
}
}
diff --git a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java
index b174b7f4ddc..33852d75e1f 100644
--- a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java
+++ b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java
@@ -40,7 +40,7 @@ import org.apache.druid.segment.column.ColumnIndexCapabilities;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
import org.apache.druid.segment.incremental.SpatialDimensionRowTransformer;
-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.SpatialIndex;
@@ -75,9 +75,12 @@ public class SpatialFilter implements Filter
return null;
}
final ColumnIndexSupplier indexSupplier = selector.getIndexSupplier(dimension);
- final SpatialIndex spatialIndex = indexSupplier == null ? null : indexSupplier.as(SpatialIndex.class);
+ if (indexSupplier == null) {
+ return new AllUnknownBitmapColumnIndex(selector);
+ }
+ final SpatialIndex spatialIndex = indexSupplier.as(SpatialIndex.class);
if (spatialIndex == null) {
- return new AllFalseBitmapColumnIndex(selector);
+ return null;
}
return new BitmapColumnIndex()
{
@@ -95,7 +98,7 @@ public class SpatialFilter implements Filter
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
Iterable<ImmutableBitmap> 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<String> 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,84 +180,103 @@ 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)
+ /**
+ * Creates a predicate-based {@link ValueMatcher} for a float-typed selector.
+ *
+ * @param selector column selector
+ * @param predicateFactory predicate to match
+ */
+ public static ValueMatcher makeFloatValueMatcher(
+ final BaseFloatColumnValueSelector selector,
+ final DruidPredicateFactory predicateFactory
+ )
{
+ final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate();
return new ValueMatcher()
{
@Override
- public boolean matches()
+ public boolean matches(boolean includeUnknown)
{
+ final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
if (selector.isNull()) {
- return false;
+ return matchNull || predicate.applyNull();
}
- return selector.getLong() == value;
+ return predicate.applyFloat(selector.getFloat());
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{
inspector.visit("selector", selector);
+ inspector.visit("predicate", predicate);
}
};
}
- public static ValueMatcher makeLongValueMatcher(
- final BaseLongColumnValueSelector selector,
- final DruidPredicateFactory predicateFactory
- )
+ /**
+ * 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)
{
- final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
return new ValueMatcher()
{
@Override
- public boolean matches()
+ public boolean matches(boolean includeUnknown)
{
if (selector.isNull()) {
- return predicate.applyNull();
+ return includeUnknown;
}
- return predicate.applyLong(selector.getLong());
+ return selector.getLong() == value;
}
@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.
+ * Creates a predicate-based {@link ValueMatcher} for a long-typed selector.
*
* @param selector column selector
* @param predicateFactory predicate to match
*/
- public static ValueMatcher makeFloatValueMatcher(
- final BaseFloatColumnValueSelector selector,
+ public static ValueMatcher makeLongValueMatcher(
+ final BaseLongColumnValueSelector selector,
final DruidPredicateFactory predicateFactory
)
{
- final DruidFloatPredicate predicate = predicateFactory.makeFloatPredicate();
+ final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
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());
+ return predicate.applyLong(selector.getLong());
}
@Override
@@ -238,6 +288,7 @@ public class ValueMatchers
};
}
+
/**
* 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<String> 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<String> 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;
+ @Nullable
+ private final ImmutableBitmap unknownBitmap;
- public AllFalseBitmapColumnIndex(ColumnIndexSelector indexSelector)
+ public AllFalseBitmapColumnIndex(BitmapFactory bitmapFactory)
{
- this(indexSelector.getBitmapFactory());
+ this(bitmapFactory, null);
}
- public AllFalseBitmapColumnIndex(BitmapFactory bitmapFactory)
+ 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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
return bitmapResultFactory.wrapAllTrue(
selector.getBitmapFactory()
diff --git a/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/AllUnknownBitmapColumnIndex.java
similarity index 67%
copy from processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java
copy to processing/src/main/java/org/apache/druid/segment/index/AllUnknownBitmapColumnIndex.java
index 686add0693c..b247face713 100644
--- a/processing/src/main/java/org/apache/druid/segment/index/AllTrueBitmapColumnIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/index/AllUnknownBitmapColumnIndex.java
@@ -24,11 +24,16 @@ import org.apache.druid.query.filter.ColumnIndexSelector;
import org.apache.druid.segment.column.ColumnIndexCapabilities;
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
-public class AllTrueBitmapColumnIndex implements BitmapColumnIndex
+/**
+ * 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 AllTrueBitmapColumnIndex(ColumnIndexSelector indexSelector)
+
+ public AllUnknownBitmapColumnIndex(ColumnIndexSelector indexSelector)
{
this.selector = indexSelector;
}
@@ -42,15 +47,18 @@ public class AllTrueBitmapColumnIndex implements BitmapColumnIndex
@Override
public double estimateSelectivity(int totalRows)
{
- return 1;
+ return 0;
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
- return bitmapResultFactory.wrapAllTrue(
- selector.getBitmapFactory()
- .complement(selector.getBitmapFactory().makeEmptyImmutableBitmap(), selector.getNumRows())
- );
+ 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> T computeBitmapResult(BitmapResultFactory<T> 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> T computeBitmapResult(BitmapResultFactory<T> 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<TDictionary extends Indexe
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (matcherFactory.isNullInputUnknown() && NullHandling.isNullOrEquivalent(dictionary.get(0))) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
diff --git a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java
index 0dde5754bbe..56bb362f346 100644
--- a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java
+++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java
@@ -106,6 +106,16 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (NullHandling.isNullOrEquivalent(dictionary.get(0))) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
@@ -119,13 +129,16 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
Predicate<String> 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<ImmutableBitmap> getBitmapIterable()
{
- final IntIntPair range = getRange(startValue, startStrict, endValue, endStrict);
- final int start = range.leftInt(), end = range.rightInt();
return () -> new Iterator<ImmutableBitmap>()
{
int currIndex = start;
@@ -169,6 +182,16 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
};
}
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (NullHandling.isNullOrEquivalent(dictionary.get(0))) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
+
private boolean applyMatcher(@Nullable final ByteBuffer valueUtf8)
{
if (valueUtf8 == null) {
diff --git a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java
index 143103cb622..9964cd225c9 100644
--- a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java
+++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8ValueIndexes.java
@@ -20,11 +20,14 @@
package org.apache.druid.segment.index;
import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators;
import com.google.common.collect.PeekingIterator;
+import org.apache.druid.annotations.SuppressFBWarnings;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.collections.bitmap.ImmutableBitmap;
+import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.ByteBufferUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprEval;
@@ -86,9 +89,13 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<TDictionary extends Indexed<ByteBuffe
return null;
}
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
@Override
public BitmapColumnIndex forSortedValues(SortedSet<String> values)
{
@@ -122,10 +130,12 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
values,
StringUtils::toUtf8ByteBuffer
),
- values.size()
+ values.size(),
+ values.contains(null)
);
}
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
@Override
public BitmapColumnIndex forSortedValuesUtf8(SortedSet<ByteBuffer> valuesUtf8)
{
@@ -138,7 +148,7 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
tailSet = valuesUtf8;
}
- return getBitmapColumnIndexForSortedIterableUtf8(tailSet, tailSet.size());
+ return getBitmapColumnIndexForSortedIterableUtf8(tailSet, tailSet.size(), valuesUtf8.contains(null));
}
private ImmutableBitmap getBitmap(int idx)
@@ -154,7 +164,7 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
/**
* Helper used by {@link #forSortedValues} and {@link #forSortedValuesUtf8}.
*/
- private BitmapColumnIndex getBitmapColumnIndexForSortedIterableUtf8(Iterable<ByteBuffer> valuesUtf8, int size)
+ private BitmapColumnIndex getBitmapColumnIndexForSortedIterableUtf8(Iterable<ByteBuffer> 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<TDictionary extends Indexed<ByteBuffe
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!valuesContainsNull && NullHandling.isNullOrEquivalent(dictionary.get(0))) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
// if the size of in-filter values is less than the threshold percentage of dictionary size, then use binary search
// based lookup per value. The algorithm works well for smaller number of values.
- return getSimpleImmutableBitmapIterableIndexFromIterator(valuesUtf8);
+ return getSimpleImmutableBitmapIterableIndexFromIterator(valuesUtf8, valuesContainsNull);
}
/**
* Iterates over the value set, using binary search to look up each element. The algorithm works well for smaller
* number of values, and must be used if the values are not sorted in the same manner as {@link #dictionary}
*/
- private SimpleImmutableBitmapIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable<ByteBuffer> valuesUtf8)
+ private SimpleImmutableBitmapIterableIndex getSimpleImmutableBitmapIterableIndexFromIterator(Iterable<ByteBuffer> valuesUtf8, boolean valuesContainsNull)
{
return new SimpleImmutableBitmapIterableIndex()
{
@@ -277,6 +297,16 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!valuesContainsNull && NullHandling.isNullOrEquivalent(dictionary.get(0))) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
diff --git a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIndex.java b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIndex.java
index c314d3c329a..ec68b3419e1 100644
--- a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIndex.java
+++ b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIndex.java
@@ -23,7 +23,8 @@ import org.apache.druid.collections.bitmap.ImmutableBitmap;
import org.apache.druid.query.BitmapResultFactory;
/**
- * {@link SimpleBitmapColumnIndex} which wraps a single {@link ImmutableBitmap}
+ * {@link SimpleBitmapColumnIndex} which wraps a single {@link ImmutableBitmap} with no 'unknowns', e.g.
+ * {@link #computeBitmapResult(BitmapResultFactory, boolean)} ignores the 'includeUnknown' argument.
*/
public final class SimpleImmutableBitmapIndex extends SimpleBitmapColumnIndex
{
@@ -41,7 +42,7 @@ public final class SimpleImmutableBitmapIndex extends SimpleBitmapColumnIndex
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<ImmutableBitmap>} in some manner
*/
@@ -35,10 +39,24 @@ public abstract class SimpleImmutableBitmapIterableIndex extends SimpleBitmapCol
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<ImmutableBitmap> 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<String> 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<String> 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<TStringDictionary extends Indexed<By
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (localDictionary.get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
@@ -373,9 +385,20 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<TStringDictionary extends Indexed<By
}
};
}
+
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!values.contains(null) && localDictionarySupplier.get().get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -529,6 +563,16 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (localDictionary.get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -594,6 +638,16 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -628,7 +682,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
if (longValue == null) {
if (inputNull) {
@@ -638,6 +692,17 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
}
final int globalId = longDictionary.indexOf(longValue);
+ 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());
}
@@ -714,6 +779,17 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!values.contains(null) && localDictionarySupplier.get().get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -823,6 +899,16 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -856,7 +942,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
if (doubleValue == null) {
if (inputNull) {
@@ -866,6 +952,17 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
}
final int globalId = doubleDictionary.indexOf(doubleValue);
+ 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());
}
@@ -942,6 +1039,17 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!values.contains(null) && localDictionarySupplier.get().get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -1035,6 +1143,16 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
+ bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -1116,6 +1234,16 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (value != null && localDictionarySupplier.get().get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
@@ -1162,6 +1290,17 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!values.contains(null) && localDictionarySupplier.get().get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
@@ -1236,6 +1375,16 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (matcherFactory.isNullInputUnknown() && localDictionary.get(0) == 0) {
+ return bitmaps.get(0);
+ }
+ return null;
+ }
};
}
}
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java
index e13b86e070d..1af97cc330d 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/NestedFieldDictionaryEncodedColumn.java
@@ -21,7 +21,6 @@
package org.apache.druid.segment.nested;
import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
import com.google.common.primitives.Doubles;
import com.google.common.primitives.Floats;
import org.apache.druid.collections.bitmap.ImmutableBitmap;
@@ -31,6 +30,8 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.query.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;
@@ -55,7 +56,6 @@ 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.historical.SingleValueHistoricalDimensionSelector;
import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
import org.apache.druid.segment.vector.BaseLongVectorValueSelector;
@@ -396,9 +396,10 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
return new ValueMatcher()
{
@Override
- public boolean matches()
+ public boolean matches(boolean includeUnknown)
{
- return getRowValue() == valueId;
+ final int rowId = getRowValue();
+ return (includeUnknown && rowId == 0) || rowId == valueId;
}
@Override
@@ -408,32 +409,48 @@ public class NestedFieldDictionaryEncodedColumn<TStringDictionary extends Indexe
}
};
} else {
- return BooleanValueMatcher.of(false);
+ return new ValueMatcher()
+ {
+ @Override
+ public boolean matches(boolean includeUnknown)
+ {
+ return includeUnknown && getRowValue() == 0;
+ }
+
+ @Override
+ public void inspectRuntimeShape(RuntimeShapeInspector inspector)
+ {
+ inspector.visit("column", NestedFieldDictionaryEncodedColumn.this);
+ }
+ };
}
} else {
// Employ caching BitSet optimization
- return makeValueMatcher(Predicates.equalTo(value));
+ return makeValueMatcher(StringPredicateDruidPredicateFactory.equalTo(value));
}
}
@Override
- public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
+ public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
{
final BitSet checkedIds = new BitSet(getCardinality());
final BitSet matchingIds = new BitSet(getCardinality());
+ final Predicate<String> 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<NestedCommon
final ExprEval<?> 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<NestedCommon
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<NestedCommon
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
if (doubleValue == null) {
if (inputNull && NullHandling.sqlCompatible()) {
@@ -322,6 +331,14 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
return bitmapResultFactory.wrapDimensionValue(getBitmap(0));
}
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());
}
@@ -401,6 +418,17 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
}
};
}
+
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!values.contains(null)) {
+ return nullValueBitmap;
+ }
+ return null;
+ }
};
}
}
@@ -451,6 +479,13 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ return nullValueBitmap;
+ }
};
}
}
@@ -522,6 +557,16 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (matcherFactory.isNullInputUnknown()) {
+ return nullValueBitmap;
+ }
+ return null;
+ }
};
}
}
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java
index d8a7e9893cf..754e7ba8948 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java
@@ -26,6 +26,7 @@ import it.unimi.dsi.fastutil.ints.IntIterator;
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;
@@ -235,7 +236,7 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
final ExprEval<?> 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<NestedCommonFo
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<NestedCommonFo
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
if (longValue == null) {
if (inputNull && NullHandling.sqlCompatible()) {
@@ -322,6 +331,14 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
return bitmapResultFactory.wrapDimensionValue(getBitmap(0));
}
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());
}
@@ -400,6 +417,17 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
}
};
}
+
+ @SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION")
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (!values.contains(null)) {
+ return nullValueBitmap;
+ }
+ return null;
+ }
};
}
}
@@ -461,6 +489,13 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ return nullValueBitmap;
+ }
};
}
}
@@ -533,6 +568,16 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
}
};
}
+
+ @Nullable
+ @Override
+ protected ImmutableBitmap getUnknownsBitmap()
+ {
+ if (matcherFactory.isNullInputUnknown()) {
+ return nullValueBitmap;
+ }
+ return null;
+ }
};
}
}
diff --git a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java
index bf7cf0605a7..1f23d663b68 100644
--- a/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java
+++ b/processing/src/main/java/org/apache/druid/segment/nested/VariantColumn.java
@@ -20,7 +20,6 @@
package org.apache.druid.segment.nested;
import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
import com.google.common.primitives.Doubles;
import com.google.common.primitives.Floats;
import it.unimi.dsi.fastutil.ints.IntArraySet;
@@ -33,6 +32,8 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.query.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;
@@ -52,7 +53,6 @@ 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.historical.SingleValueHistoricalDimensionSelector;
import org.apache.druid.segment.vector.BaseDoubleVectorValueSelector;
import org.apache.druid.segment.vector.MultiValueDimensionVectorSelector;
@@ -477,9 +477,11 @@ public class VariantColumn<TStringDictionary extends Indexed<ByteBuffer>>
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<TStringDictionary extends Indexed<ByteBuffer>>
}
};
} 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<String> predicate)
+ public ValueMatcher makeValueMatcher(final DruidPredicateFactory predicateFactory)
{
final BitSet checkedIds = new BitSet(getCardinality());
final BitSet matchingIds = new BitSet(getCardinality());
+ final Predicate<String> 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<NestedCommonForma
ExpressionType.fromColumnTypeStrict(logicalType)
);
if (castForComparison == null) {
- return new AllFalseBitmapColumnIndex(bitmapFactory);
+ return new AllFalseBitmapColumnIndex(bitmapFactory, nullValueBitmap);
}
final Object[] arrayToMatch = castForComparison.asArray();
Indexed elements;
@@ -362,7 +363,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
}
if (ids[i] < 0) {
if (value == null) {
- return new AllFalseBitmapColumnIndex(bitmapFactory);
+ return new AllFalseBitmapColumnIndex(bitmapFactory, nullValueBitmap);
}
}
}
@@ -381,9 +382,17 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<NestedCommonForma
ExpressionType.fromColumnTypeStrict(logicalType.getElementType())
);
if (castForComparison == null) {
- return new AllFalseBitmapColumnIndex(bitmapFactory);
+ return new AllFalseBitmapColumnIndex(bitmapFactory, nullValueBitmap);
}
Indexed elements;
final int elementOffset;
@@ -443,10 +452,17 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<String> 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<Map<String, Object>> 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<String> 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<String> 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<String> predicate = predicateFactory.makeStringPredicate();
if (evaluated.isArray()) {
List<String> 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<ImmutableBitmap> 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<ImmutableBitmap> 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<ImmutableBitmap> getBitmapsInRange(
+ DictionaryEncodedStringValueIndex delegate,
+ IdMapping idMapping,
+ Predicate<String> matcher,
+ int start,
+ int end
+ )
+ {
+ return () -> new Iterator<ImmutableBitmap>()
+ {
+ 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<ImmutableBitmap> getBitmapsInRange(Predicate<String> matcher, int start, int end)
{
- return () -> new Iterator<ImmutableBitmap>()
- {
- 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<ImmutableBitmap> nullValueBitmapSupplier;
- private ListFilteredNullValueIndex(DictionaryEncodedStringValueIndex delegate, IdMapping idMapping, int numRows)
+ private ListFilteredNullValueIndex(Supplier<ImmutableBitmap> 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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
- {
- return bitmapResultFactory.complement(
- bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable()),
- numRows
+ return Filters.estimateSelectivity(
+ Collections.singletonList(nullValueBitmapSupplier.get()).iterator(),
+ totalRows
);
}
@Override
- protected Iterable<ImmutableBitmap> getBitmapIterable()
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<ImmutableBitmap> nullValueBitmapSupplier;
private ListFilteredStringValueSetIndexes(
DictionaryEncodedStringValueIndex delegate,
- IdMapping idMapping
+ IdMapping idMapping,
+ Supplier<ImmutableBitmap> nullValueBitmapSupplier
)
{
super(delegate, idMapping);
+ this.nullValueBitmapSupplier = nullValueBitmapSupplier;
}
@Override
@@ -433,8 +460,13 @@ public class ListFilteredVirtualColumn implements VirtualColumn
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> 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<ImmutableBitmap> nullValueBitmapSupplier;
- private ListFilteredDruidPredicateIndexes(DictionaryEncodedStringValueIndex delegate, IdMapping idMapping)
+ private ListFilteredDruidPredicateIndexes(
+ DictionaryEncodedStringValueIndex delegate,
+ IdMapping idMapping,
+ Supplier<ImmutableBitmap> nullValueBitmapSupplier
+ )
{
super(delegate, idMapping);
+ this.nullValueBitmapSupplier = nullValueBitmapSupplier;
}
@Override
@@ -524,12 +569,19 @@ public class ListFilteredVirtualColumn implements VirtualColumn
}
@Override
- public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory)
+ public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
final int start = 0, end = getCardinality();
- return bitmapResultFactory.unionDimensionValueBitmaps(
- getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end)
- );
+ Iterable<ImmutableBitmap> 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<ImmutableBitmap> nullValueBitmapSupplier;
private ListFilteredLexicographicalRangeIndexes(
DictionaryEncodedStringValueIndex delegate,
- IdMapping idMapping
+ IdMapping idMapping,
+ Supplier<ImmutableBitmap> 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<String> 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<String> 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<Integer> 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<Integer> expected, ImmutableConciseSet set, int endIndex)
{
List<Integer> 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<String> 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<String> 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<String> 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<String> 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<String> 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<Result<TimeseriesResultValue>> 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 <T> void assertExpectedResults(Iterable<Result<T>> expectedResults, Iterable<Result<T>> results)
+ protected <T> void assertExpectedResults(Iterable<Result<T>> expectedResults, Iterable<Result<T>> results)
{
if (descending) {
expectedResults = TestHelper.revert(expectedResults);
@@ -151,7 +151,7 @@ public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest
protected final QueryRunner<Result<TimeseriesResultValue>> runner;
protected final boolean descending;
protected final boolean vectorize;
- private final List<AggregatorFactory> aggregatorFactoryList;
+ protected final List<AggregatorFactory> aggregatorFactoryList;
public TimeseriesQueryRunnerTest(
QueryRunner<Result<TimeseriesResultValue>> runner,
@@ -1767,30 +1767,49 @@ public class TimeseriesQueryRunnerTest extends InitializedNullHandlingTest
.context(makeContext())
.build();
- List<Result<TimeseriesResultValue>> 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<Result<TimeseriesResultValue>> 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<Result<TimeseriesResultValue>> 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<String> 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<Integer> 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<String> 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.<DimensionSchema>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<Map<String, Object>> parser,
final RowSignature signature,
@Nullable Object... elements
@@ -1034,7 +1048,7 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
final List<String> 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<InputRow> ROWS = ImmutableList.<InputRow>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<InputRow> ROWS = ImmutableList.<Map<String, Object>>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<InputRow> 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<SelectorFilter, Filter> 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<InputRow> ROWS = ImmutableList.<InputRow>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<InputRow> 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<Map<String, Object>> 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<InputRow> 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<IndexBuilder, Pair<StorageAdapter, Closeable>> 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<IndexBuilder, Pair<StorageAdapter, Closeable>> 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
... 2830 lines suppressed ...
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org