mirror of https://github.com/apache/druid.git
sql compatible three-valued logic native filters (#15058)
* sql compatible tri-state native logical filters when druid.expressions.useStrictBooleans=true and druid.generic.useDefaultValueForNull=false, and new druid.generic.useThreeValueLogicForNativeFilters=true * log.warn if non-default configurations are used to guide operators towards SQL complaint behavior
This commit is contained in:
parent
265c811963
commit
d0f64608eb
|
@ -798,8 +798,9 @@ Support for 64-bit floating point columns was released in Druid 0.11.0, so if yo
|
|||
Prior to version 0.13.0, Druid string columns treated `''` and `null` values as interchangeable, and numeric columns were unable to represent `null` values, coercing `null` to `0`. Druid 0.13.0 introduced a mode which enabled SQL compatible null handling, allowing string columns to distinguish empty strings from nulls, and numeric columns to contain null rows.
|
||||
|
||||
|Property|Description|Default|
|
||||
|---|---|---|
|
||||
|--------|-----------|-------|
|
||||
|`druid.generic.useDefaultValueForNull`|Set to `false` to store and query data in SQL compatible mode. When set to `true` (legacy mode), `null` values will be stored as `''` for string columns and `0` for numeric columns.|`false`|
|
||||
|`druid.generic.useThreeValueLogicForNativeFilters`|Set to `true` to use SQL compatible three-value logic when processing native Druid filters when `druid.generic.useDefaultValueForNull=false` and `druid.expressions.useStrictBooleans=true`. When set to `false` Druid uses 2 value logic for filter processing, even when `druid.generic.useDefaultValueForNull=false` and `druid.expressions.useStrictBooleans=true`. See [boolean handling](../querying/sql-data-types.md#boolean-logic) for more details|`true`|
|
||||
|`druid.generic.ignoreNullsForStringCardinality`|When set to `true`, `null` values will be ignored for the built-in cardinality aggregator over string columns. Set to `false` to include `null` values while estimating cardinality of only string columns using the built-in cardinality aggregator. This setting takes effect only when `druid.generic.useDefaultValueForNull` is set to `true` and is ignored in SQL compatibility mode. Additionally, empty strings (equivalent to null) are not counted when this is set to `true`. |`false`|
|
||||
This mode does have a storage size and query performance cost, see [segment documentation](../design/segments.md#handling-null-values) for more details.
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -279,7 +279,7 @@ public class DoublesSketchSqlAggregatorTest extends BaseCalciteQueryTest
|
|||
10.1,
|
||||
20.2,
|
||||
Double.NaN,
|
||||
10.1,
|
||||
2.0,
|
||||
Double.NaN
|
||||
}
|
||||
);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -793,7 +793,11 @@ public class MSQSelectTest extends MSQTestBase
|
|||
.build())
|
||||
.setExpectedRowSignature(rowSignature)
|
||||
.setExpectedResultRows(
|
||||
ImmutableList.of(
|
||||
NullHandling.sqlCompatible()
|
||||
? ImmutableList.of(
|
||||
new Object[]{"xabc", 1L}
|
||||
)
|
||||
: ImmutableList.of(
|
||||
new Object[]{NullHandling.defaultStringValue(), 3L},
|
||||
new Object[]{"xabc", 1L}
|
||||
)
|
||||
|
|
|
@ -22,6 +22,7 @@ package org.apache.druid.common.config;
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.math.expr.ExpressionProcessing;
|
||||
import org.apache.druid.segment.column.ValueType;
|
||||
import org.apache.druid.segment.data.Indexed;
|
||||
|
||||
|
@ -61,13 +62,19 @@ public class NullHandling
|
|||
@VisibleForTesting
|
||||
public static void initializeForTests()
|
||||
{
|
||||
INSTANCE = new NullValueHandlingConfig(null, null);
|
||||
INSTANCE = new NullValueHandlingConfig(null, null, null);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void initializeForTestsWithValues(Boolean useDefForNull, Boolean ignoreNullForString)
|
||||
{
|
||||
INSTANCE = new NullValueHandlingConfig(useDefForNull, ignoreNullForString);
|
||||
initializeForTestsWithValues(useDefForNull, null, ignoreNullForString);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void initializeForTestsWithValues(Boolean useDefForNull, Boolean useThreeValueLogic, Boolean ignoreNullForString)
|
||||
{
|
||||
INSTANCE = new NullValueHandlingConfig(useDefForNull, useThreeValueLogic, ignoreNullForString);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -99,6 +106,13 @@ public class NullHandling
|
|||
return !replaceWithDefault();
|
||||
}
|
||||
|
||||
public static boolean useThreeValueLogic()
|
||||
{
|
||||
return NullHandling.sqlCompatible() &&
|
||||
INSTANCE.isUseThreeValueLogicForNativeFilters() &&
|
||||
ExpressionProcessing.useStrictBooleans();
|
||||
}
|
||||
|
||||
@Nullable
|
||||
public static String nullToEmptyIfNeeded(@Nullable String value)
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -35,7 +35,7 @@ public class FilteredAggregator implements Aggregator
|
|||
@Override
|
||||
public void aggregate()
|
||||
{
|
||||
if (matcher.matches()) {
|
||||
if (matcher.matches(false)) {
|
||||
delegate.aggregate();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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
|
||||
*
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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()
|
||||
{
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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])) {
|
||||
|
|
|
@ -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])) {
|
||||
|
|
|
@ -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])) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -318,7 +318,7 @@ public class LazilyDecoratedRowsAndColumns implements RowsAndColumns
|
|||
continue;
|
||||
}
|
||||
|
||||
if (!matcher.matches()) {
|
||||
if (!matcher.matches(false)) {
|
||||
rowsToSkip.set(theId);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()) {
|
||||
|
|
|
@ -202,6 +202,7 @@ public class IndexMergerV9 implements IndexMerger
|
|||
mergers.add(
|
||||
handler.makeMerger(
|
||||
indexSpec,
|
||||
|
||||
segmentWriteOutMedium,
|
||||
dimFormats.get(i).toColumnCapabilities(),
|
||||
progress,
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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++;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -20,17 +20,34 @@
|
|||
package org.apache.druid.segment.filter;
|
||||
|
||||
import org.apache.druid.query.filter.ValueMatcher;
|
||||
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
|
||||
/**
|
||||
*/
|
||||
public final class BooleanValueMatcher
|
||||
* Constant condition {@link ValueMatcher} that always returns the boolean value passed to {@link #matches(boolean)},
|
||||
* for columns which contain only null values
|
||||
*/
|
||||
final class AllUnknownValueMatcher implements ValueMatcher
|
||||
{
|
||||
public static ValueMatcher of(boolean matches)
|
||||
private static final AllUnknownValueMatcher INSTANCE = new AllUnknownValueMatcher();
|
||||
|
||||
public static AllUnknownValueMatcher instance()
|
||||
{
|
||||
return matches ? TrueValueMatcher.instance() : FalseValueMatcher.instance();
|
||||
return INSTANCE;
|
||||
}
|
||||
|
||||
private BooleanValueMatcher()
|
||||
private AllUnknownValueMatcher()
|
||||
{
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
return includeUnknown;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
// nothing to inspect
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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()][];
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,63 +180,6 @@ public class ValueMatchers
|
|||
};
|
||||
}
|
||||
|
||||
public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, final String value)
|
||||
{
|
||||
final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
|
||||
if (matchVal == null) {
|
||||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
return makeLongValueMatcher(selector, matchVal);
|
||||
}
|
||||
|
||||
public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, long value)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
}
|
||||
return selector.getLong() == value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
public static ValueMatcher makeLongValueMatcher(
|
||||
final BaseLongColumnValueSelector selector,
|
||||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
}
|
||||
return predicate.applyLong(selector.getLong());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a predicate-based {@link ValueMatcher} for a float-typed selector.
|
||||
*
|
||||
|
@ -221,10 +195,11 @@ public class ValueMatchers
|
|||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
return matchNull || predicate.applyNull();
|
||||
}
|
||||
return predicate.applyFloat(selector.getFloat());
|
||||
}
|
||||
|
@ -238,6 +213,82 @@ public class ValueMatchers
|
|||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a long-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param value value to match
|
||||
*/
|
||||
public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, final String value)
|
||||
{
|
||||
final Long matchVal = DimensionHandlerUtils.convertObjectToLong(value);
|
||||
if (matchVal == null) {
|
||||
return makeNumericNullValueMatcher(selector);
|
||||
}
|
||||
return makeLongValueMatcher(selector, matchVal);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a long-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param value value to match
|
||||
*/
|
||||
public static ValueMatcher makeLongValueMatcher(final BaseLongColumnValueSelector selector, long value)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return includeUnknown;
|
||||
}
|
||||
return selector.getLong() == value;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a predicate-based {@link ValueMatcher} for a long-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param predicateFactory predicate to match
|
||||
*/
|
||||
public static ValueMatcher makeLongValueMatcher(
|
||||
final BaseLongColumnValueSelector selector,
|
||||
final DruidPredicateFactory predicateFactory
|
||||
)
|
||||
{
|
||||
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
if (selector.isNull()) {
|
||||
return matchNull || predicate.applyNull();
|
||||
}
|
||||
return predicate.applyLong(selector.getLong());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
inspector.visit("predicate", predicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a double-typed selector.
|
||||
*
|
||||
|
@ -257,6 +308,12 @@ public class ValueMatchers
|
|||
return makeDoubleValueMatcher(selector, matchVal);
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a constant-based {@link ValueMatcher} for a double-typed selector.
|
||||
*
|
||||
* @param selector column selector
|
||||
* @param value value to match
|
||||
*/
|
||||
public static ValueMatcher makeDoubleValueMatcher(
|
||||
final BaseDoubleColumnValueSelector selector,
|
||||
final double value
|
||||
|
@ -267,10 +324,10 @@ public class ValueMatchers
|
|||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
if (selector.isNull()) {
|
||||
return false;
|
||||
return includeUnknown;
|
||||
}
|
||||
return Double.doubleToLongBits(selector.getDouble()) == matchValLongBits;
|
||||
}
|
||||
|
@ -298,10 +355,11 @@ public class ValueMatchers
|
|||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
final boolean matchNull = includeUnknown && predicateFactory.isNullInputUnknown();
|
||||
if (selector.isNull()) {
|
||||
return predicate.applyNull();
|
||||
return matchNull || predicate.applyNull();
|
||||
}
|
||||
return predicate.applyDouble(selector.getDouble());
|
||||
}
|
||||
|
@ -315,6 +373,140 @@ public class ValueMatchers
|
|||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a matcher that should always return false, except when {@code includeUnknown} is set, in which case only
|
||||
* null values will be matched. This is typically used when the filter should never match any actual values, but
|
||||
* still needs to be able to report 'unknown' matches.
|
||||
*/
|
||||
public static ValueMatcher makeAlwaysFalseDimensionMatcher(final DimensionSelector selector, boolean multiValue)
|
||||
{
|
||||
final IdLookup lookup = selector.idLookup();
|
||||
// if the column doesn't have null
|
||||
if (lookup == null || !selector.nameLookupPossibleInAdvance()) {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
if (includeUnknown) {
|
||||
IndexedInts row = selector.getRow();
|
||||
final int size = row.size();
|
||||
if (size == 0) {
|
||||
return true;
|
||||
}
|
||||
for (int i = 0; i < size; i++) {
|
||||
if (NullHandling.isNullOrEquivalent(selector.lookupName(row.get(i)))) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
final int nullId = lookup.lookupId(null);
|
||||
if (nullId < 0) {
|
||||
// column doesn't have null value so no unknowns, can safely return always false matcher
|
||||
return ValueMatchers.allFalse();
|
||||
}
|
||||
if (multiValue) {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
if (includeUnknown) {
|
||||
IndexedInts row = selector.getRow();
|
||||
final int size = row.size();
|
||||
if (size == 0) {
|
||||
return true;
|
||||
}
|
||||
for (int i = 0; i < size; i++) {
|
||||
if (row.get(i) == nullId) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
return includeUnknown && selector.getRow().get(0) == nullId;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a matcher that should always return false, except when {@code includeUnknown} is set, in which case only
|
||||
* null values will be matched. This is typically used when the filter should never match any actual values, but
|
||||
* still needs to be able to report 'unknown' matches.
|
||||
*/
|
||||
public static ValueMatcher makeAlwaysFalseNumericMatcher(BaseNullableColumnValueSelector selector)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
return includeUnknown && selector.isNull();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a matcher that should always return false, except when {@code includeUnknown} is set, in which case only
|
||||
* null values will be matched. This is typically used when the filter should never match any actual values, but
|
||||
* still needs to be able to report 'unknown' matches.
|
||||
*/
|
||||
public static ValueMatcher makeAlwaysFalseObjectMatcher(BaseObjectColumnValueSelector<?> selector)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches(boolean includeUnknown)
|
||||
{
|
||||
return includeUnknown && selector.getObject() == null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* If applying {@code predicate} to {@code selector} would always return a constant, returns that constant.
|
||||
* Otherwise, returns null.
|
||||
|
@ -327,7 +519,7 @@ public class ValueMatchers
|
|||
* @param predicate predicate to apply
|
||||
*/
|
||||
@Nullable
|
||||
public static Boolean toBooleanIfPossible(
|
||||
public static ConstantMatcherType toConstantMatcherTypeIfPossible(
|
||||
final DimensionDictionarySelector selector,
|
||||
final boolean hasMultipleValues,
|
||||
final Predicate<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();
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -20,23 +20,28 @@
|
|||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
||||
import org.apache.druid.collections.bitmap.ImmutableBitmap;
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.filter.ColumnIndexSelector;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
public class AllFalseBitmapColumnIndex implements BitmapColumnIndex
|
||||
{
|
||||
private final BitmapFactory bitmapFactory;
|
||||
|
||||
public AllFalseBitmapColumnIndex(ColumnIndexSelector indexSelector)
|
||||
{
|
||||
this(indexSelector.getBitmapFactory());
|
||||
}
|
||||
@Nullable
|
||||
private final ImmutableBitmap unknownBitmap;
|
||||
|
||||
public AllFalseBitmapColumnIndex(BitmapFactory bitmapFactory)
|
||||
{
|
||||
this(bitmapFactory, null);
|
||||
}
|
||||
|
||||
public AllFalseBitmapColumnIndex(BitmapFactory bitmapFactory, @Nullable ImmutableBitmap unknownBitmap)
|
||||
{
|
||||
this.bitmapFactory = bitmapFactory;
|
||||
this.unknownBitmap = unknownBitmap;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -52,8 +57,11 @@ public class AllFalseBitmapColumnIndex implements BitmapColumnIndex
|
|||
}
|
||||
|
||||
@Override
|
||||
public <T> 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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -0,0 +1,64 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.segment.index;
|
||||
|
||||
import org.apache.druid.query.BitmapResultFactory;
|
||||
import org.apache.druid.query.filter.ColumnIndexSelector;
|
||||
import org.apache.druid.segment.column.ColumnIndexCapabilities;
|
||||
import org.apache.druid.segment.column.SimpleColumnIndexCapabilities;
|
||||
|
||||
/**
|
||||
* Like {@link AllFalseBitmapColumnIndex} during normal operation, except if called with 'includeUnknowns' becomes like
|
||||
* {@link AllTrueBitmapColumnIndex}.
|
||||
*/
|
||||
public class AllUnknownBitmapColumnIndex implements BitmapColumnIndex
|
||||
{
|
||||
private final ColumnIndexSelector selector;
|
||||
|
||||
|
||||
public AllUnknownBitmapColumnIndex(ColumnIndexSelector indexSelector)
|
||||
{
|
||||
this.selector = indexSelector;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ColumnIndexCapabilities getIndexCapabilities()
|
||||
{
|
||||
return SimpleColumnIndexCapabilities.getConstant();
|
||||
}
|
||||
|
||||
@Override
|
||||
public double estimateSelectivity(int totalRows)
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
|
||||
{
|
||||
if (includeUnknown) {
|
||||
return bitmapResultFactory.wrapAllTrue(
|
||||
selector.getBitmapFactory()
|
||||
.complement(selector.getBitmapFactory().makeEmptyImmutableBitmap(), selector.getNumRows())
|
||||
);
|
||||
}
|
||||
return bitmapResultFactory.wrapAllFalse(selector.getBitmapFactory().makeEmptyImmutableBitmap());
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue