mirror of https://github.com/apache/druid.git
Add ExpressionFilter. (#4405)
* Add ExpressionFilter. The expression filter expects a single argument, "expression", and matches rows where that expression is true. * Code review comments. * CR comment. * Fix logic. * Fix test. * Remove unused import.
This commit is contained in:
parent
b333deae9d
commit
679cf277c0
|
@ -200,6 +200,12 @@ public class BoundFilterBenchmark
|
|||
return dictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
|
|
|
@ -154,6 +154,12 @@ public class DimensionPredicateFilterBenchmark
|
|||
return dictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
|
|
|
@ -161,6 +161,12 @@ public class LikeFilterBenchmark
|
|||
return dictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,32 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.expression;
|
||||
|
||||
import io.druid.math.expr.Expr;
|
||||
|
||||
public class ExprUtils
|
||||
{
|
||||
private static final Expr.ObjectBinding NIL_BINDINGS = name -> null;
|
||||
|
||||
public static Expr.ObjectBinding nilBindings()
|
||||
{
|
||||
return NIL_BINDINGS;
|
||||
}
|
||||
}
|
|
@ -30,6 +30,7 @@ import io.druid.segment.data.Indexed;
|
|||
public interface BitmapIndexSelector
|
||||
{
|
||||
public Indexed<String> getDimensionValues(String dimension);
|
||||
public boolean hasMultipleValues(String dimension);
|
||||
public int getNumRows();
|
||||
public BitmapFactory getBitmapFactory();
|
||||
public BitmapIndex getBitmapIndex(String dimension);
|
||||
|
|
|
@ -41,7 +41,8 @@ import io.druid.java.util.common.Cacheable;
|
|||
@JsonSubTypes.Type(name="in", value=InDimFilter.class),
|
||||
@JsonSubTypes.Type(name="bound", value=BoundDimFilter.class),
|
||||
@JsonSubTypes.Type(name="interval", value=IntervalDimFilter.class),
|
||||
@JsonSubTypes.Type(name="like", value=LikeDimFilter.class)
|
||||
@JsonSubTypes.Type(name="like", value=LikeDimFilter.class),
|
||||
@JsonSubTypes.Type(name="expression", value=ExpressionDimFilter.class)
|
||||
})
|
||||
public interface DimFilter extends Cacheable
|
||||
{
|
||||
|
|
|
@ -51,6 +51,7 @@ public class DimFilterUtils
|
|||
static final byte INTERVAL_CACHE_ID = 0xB;
|
||||
static final byte LIKE_CACHE_ID = 0xC;
|
||||
static final byte COLUMN_COMPARISON_CACHE_ID = 0xD;
|
||||
static final byte EXPRESSION_CACHE_ID = 0xE;
|
||||
public static final byte STRING_SEPARATOR = (byte) 0xFF;
|
||||
|
||||
static byte[] computeCacheKey(byte cacheIdKey, List<DimFilter> filters)
|
||||
|
|
|
@ -0,0 +1,107 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.filter;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.collect.RangeSet;
|
||||
import io.druid.math.expr.Expr;
|
||||
import io.druid.math.expr.ExprMacroTable;
|
||||
import io.druid.math.expr.Parser;
|
||||
import io.druid.query.cache.CacheKeyBuilder;
|
||||
import io.druid.segment.filter.ExpressionFilter;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
public class ExpressionDimFilter implements DimFilter
|
||||
{
|
||||
private final String expression;
|
||||
private final Expr parsed;
|
||||
|
||||
@JsonCreator
|
||||
public ExpressionDimFilter(
|
||||
@JsonProperty("expression") final String expression,
|
||||
@JacksonInject ExprMacroTable macroTable
|
||||
)
|
||||
{
|
||||
this.expression = expression;
|
||||
this.parsed = Parser.parse(expression, macroTable);
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public String getExpression()
|
||||
{
|
||||
return expression;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DimFilter optimize()
|
||||
{
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Filter toFilter()
|
||||
{
|
||||
return new ExpressionFilter(parsed);
|
||||
}
|
||||
|
||||
@Override
|
||||
public RangeSet<String> getDimensionRangeSet(final String dimension)
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
return new CacheKeyBuilder(DimFilterUtils.EXPRESSION_CACHE_ID)
|
||||
.appendString(expression)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(final Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
final ExpressionDimFilter that = (ExpressionDimFilter) o;
|
||||
return Objects.equals(expression, that.expression);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(expression);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "ExpressionDimFilter{" +
|
||||
"expression='" + expression + '\'' +
|
||||
'}';
|
||||
}
|
||||
}
|
|
@ -34,7 +34,9 @@ import io.druid.segment.ColumnSelectorFactory;
|
|||
public interface Filter
|
||||
{
|
||||
/**
|
||||
* Get a bitmap index, indicating rows that match this filter.
|
||||
* Get a bitmap index, indicating rows that match this filter. Do not call this method unless
|
||||
* {@link #supportsBitmapIndex(BitmapIndexSelector)} returns true. Behavior in the case that
|
||||
* {@link #supportsBitmapIndex(BitmapIndexSelector)} returns false is undefined.
|
||||
*
|
||||
* @param selector Object used to retrieve bitmap indexes
|
||||
*
|
||||
|
@ -47,6 +49,17 @@ public interface Filter
|
|||
return getBitmapResult(selector, new DefaultBitmapResultFactory(selector.getBitmapFactory()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a (possibly wrapped) bitmap index, indicating rows that match this filter. Do not call this method unless
|
||||
* {@link #supportsBitmapIndex(BitmapIndexSelector)} returns true. Behavior in the case that
|
||||
* {@link #supportsBitmapIndex(BitmapIndexSelector)} returns false is undefined.
|
||||
*
|
||||
* @param selector Object used to retrieve bitmap indexes
|
||||
*
|
||||
* @return A bitmap indicating rows that match this filter.
|
||||
*
|
||||
* @see Filter#estimateSelectivity(BitmapIndexSelector)
|
||||
*/
|
||||
default <T> T getBitmapResult(BitmapIndexSelector selector, BitmapResultFactory<T> bitmapResultFactory)
|
||||
{
|
||||
return bitmapResultFactory.wrapUnknown(getBitmapIndex(selector));
|
||||
|
|
|
@ -27,7 +27,6 @@ import io.druid.query.filter.BitmapIndexSelector;
|
|||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.DictionaryEncodedColumn;
|
||||
import io.druid.segment.column.GenericColumn;
|
||||
import io.druid.segment.column.ValueType;
|
||||
|
@ -59,7 +58,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public Indexed<String> getDimensionValues(String dimension)
|
||||
{
|
||||
if (isFilterableVirtualColumn(dimension)) {
|
||||
if (isVirtualColumn(dimension)) {
|
||||
// Virtual columns don't have dictionaries or indexes.
|
||||
return null;
|
||||
}
|
||||
|
@ -109,6 +108,17 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
{
|
||||
if (isVirtualColumn(dimension)) {
|
||||
return virtualColumns.getVirtualColumn(dimension).capabilities(dimension).hasMultipleValues();
|
||||
}
|
||||
|
||||
final Column column = index.getColumn(dimension);
|
||||
return column != null && column.getCapabilities().hasMultipleValues();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
|
@ -126,7 +136,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public BitmapIndex getBitmapIndex(String dimension)
|
||||
{
|
||||
if (isFilterableVirtualColumn(dimension)) {
|
||||
if (isVirtualColumn(dimension)) {
|
||||
// Virtual columns don't have dictionaries or indexes.
|
||||
return null;
|
||||
}
|
||||
|
@ -193,7 +203,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, String value)
|
||||
{
|
||||
if (isFilterableVirtualColumn(dimension)) {
|
||||
if (isVirtualColumn(dimension)) {
|
||||
// Virtual columns don't have dictionaries or indexes.
|
||||
return null;
|
||||
}
|
||||
|
@ -218,7 +228,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
@Override
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
if (isFilterableVirtualColumn(dimension)) {
|
||||
if (isVirtualColumn(dimension)) {
|
||||
return new ImmutableRTree();
|
||||
}
|
||||
|
||||
|
@ -230,14 +240,9 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
return column.getSpatialIndex().getRTree();
|
||||
}
|
||||
|
||||
private boolean isFilterableVirtualColumn(final String columnName)
|
||||
private boolean isVirtualColumn(final String columnName)
|
||||
{
|
||||
final ColumnCapabilities columnCapabilities = virtualColumns.getColumnCapabilities(columnName);
|
||||
if (columnCapabilities == null) {
|
||||
return false;
|
||||
} else {
|
||||
return Filters.FILTERABLE_TYPES.contains(columnCapabilities.getType());
|
||||
}
|
||||
return virtualColumns.getVirtualColumn(columnName) != null;
|
||||
}
|
||||
|
||||
private static boolean columnSupportsFiltering(Column column)
|
||||
|
|
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.segment.filter;
|
||||
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.math.expr.Evals;
|
||||
import io.druid.math.expr.Expr;
|
||||
import io.druid.math.expr.Parser;
|
||||
import io.druid.query.BitmapResultFactory;
|
||||
import io.druid.query.expression.ExprUtils;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.ValueMatcher;
|
||||
import io.druid.query.monomorphicprocessing.RuntimeShapeInspector;
|
||||
import io.druid.segment.ColumnSelector;
|
||||
import io.druid.segment.ColumnSelectorFactory;
|
||||
import io.druid.segment.LongColumnSelector;
|
||||
import io.druid.segment.virtual.ExpressionSelectors;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
public class ExpressionFilter implements Filter
|
||||
{
|
||||
private final Expr expr;
|
||||
private final Set<String> requiredBindings;
|
||||
|
||||
public ExpressionFilter(final Expr expr)
|
||||
{
|
||||
this.expr = expr;
|
||||
this.requiredBindings = ImmutableSet.copyOf(Parser.findRequiredBindings(expr));
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(final ColumnSelectorFactory factory)
|
||||
{
|
||||
final LongColumnSelector selector = ExpressionSelectors.makeLongColumnSelector(factory, expr, 0L);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return Evals.asBoolean(selector.get());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void inspectRuntimeShape(final RuntimeShapeInspector inspector)
|
||||
{
|
||||
inspector.visit("selector", selector);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsBitmapIndex(final BitmapIndexSelector selector)
|
||||
{
|
||||
if (requiredBindings.isEmpty()) {
|
||||
// Constant expression.
|
||||
return true;
|
||||
} else if (requiredBindings.size() == 1) {
|
||||
// Single-column expression. We can use bitmap indexes if this column has an index and does not have
|
||||
// multiple values. The lack of multiple values is important because expression filters treat multi-value
|
||||
// arrays as nulls, which doesn't permit index based filtering.
|
||||
final String column = Iterables.getOnlyElement(requiredBindings);
|
||||
return selector.getBitmapIndex(column) != null && !selector.hasMultipleValues(column);
|
||||
} else {
|
||||
// Multi-column expression.
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T getBitmapResult(final BitmapIndexSelector selector, final BitmapResultFactory<T> bitmapResultFactory)
|
||||
{
|
||||
if (requiredBindings.isEmpty()) {
|
||||
// Constant expression.
|
||||
if (expr.eval(ExprUtils.nilBindings()).asBoolean()) {
|
||||
return bitmapResultFactory.wrapAllTrue(Filters.allTrue(selector));
|
||||
} else {
|
||||
return bitmapResultFactory.wrapAllFalse(Filters.allFalse(selector));
|
||||
}
|
||||
} else {
|
||||
// Can assume there's only one binding and it has a bitmap index, otherwise supportsBitmapIndex would have
|
||||
// returned false and the caller should not have called us.
|
||||
final String column = Iterables.getOnlyElement(requiredBindings);
|
||||
return Filters.matchPredicate(
|
||||
column,
|
||||
selector,
|
||||
bitmapResultFactory,
|
||||
value -> expr.eval(identifierName -> {
|
||||
// There's only one binding, and it must be the single column, so it can safely be ignored in production.
|
||||
assert column.equals(identifierName);
|
||||
return value;
|
||||
}).asBoolean()
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsSelectivityEstimation(
|
||||
final ColumnSelector columnSelector,
|
||||
final BitmapIndexSelector indexSelector
|
||||
)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public double estimateSelectivity(final BitmapIndexSelector indexSelector)
|
||||
{
|
||||
// Selectivity estimation not supported.
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,207 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.FloatDimensionSchema;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.LongDimensionSchema;
|
||||
import io.druid.data.input.impl.MapInputRowParser;
|
||||
import io.druid.data.input.impl.StringDimensionSchema;
|
||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.query.expression.TestExprMacroTable;
|
||||
import io.druid.query.filter.ExpressionDimFilter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class ExpressionFilterTest extends BaseFilterTest
|
||||
{
|
||||
private static final String TIMESTAMP_COLUMN = "timestamp";
|
||||
|
||||
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||
new TimeAndDimsParseSpec(
|
||||
new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")),
|
||||
new DimensionsSpec(
|
||||
ImmutableList.of(
|
||||
new StringDimensionSchema("dim0"),
|
||||
new LongDimensionSchema("dim1"),
|
||||
new FloatDimensionSchema("dim2"),
|
||||
new StringDimensionSchema("dim3"),
|
||||
new StringDimensionSchema("dim4")
|
||||
),
|
||||
null,
|
||||
null
|
||||
)
|
||||
)
|
||||
);
|
||||
|
||||
private static final List<InputRow> ROWS = ImmutableList.<Map<String, Object>>of(
|
||||
ImmutableMap.of("dim0", "0", "dim1", 0L, "dim2", 0.0f, "dim3", "", "dim4", ImmutableList.of("1", "2")),
|
||||
ImmutableMap.of("dim0", "1", "dim1", 1L, "dim2", 1.0f, "dim3", "10", "dim4", ImmutableList.of()),
|
||||
ImmutableMap.of("dim0", "2", "dim1", 2L, "dim2", 2.0f, "dim3", "2", "dim4", ImmutableList.of("")),
|
||||
ImmutableMap.of("dim0", "3", "dim1", 3L, "dim2", 3.0f, "dim3", "1", "dim4", ImmutableList.of("3")),
|
||||
ImmutableMap.of("dim0", "4", "dim1", 4L, "dim2", 4.0f, "dim3", "1", "dim4", ImmutableList.of("4", "5")),
|
||||
ImmutableMap.of("dim0", "5", "dim1", 5L, "dim2", 5.0f, "dim3", "5", "dim4", ImmutableList.of("4", "5")),
|
||||
ImmutableMap.of("dim0", "6", "dim1", 6L, "dim2", 6.0f, "dim3", "1"),
|
||||
ImmutableMap.of("dim0", "7", "dim1", 7L, "dim2", 7.0f, "dim3", "a"),
|
||||
ImmutableMap.of("dim0", "8", "dim1", 8L, "dim2", 8.0f, "dim3", 8L),
|
||||
ImmutableMap.of("dim0", "9", "dim1", 9L, "dim2", 9.0f, "dim3", 1.234f, "dim4", 1.234f)
|
||||
).stream().map(PARSER::parse).collect(Collectors.toList());
|
||||
|
||||
public ExpressionFilterTest(
|
||||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(
|
||||
testName,
|
||||
ROWS,
|
||||
indexBuilder.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withDimensionsSpec(PARSER.getParseSpec().getDimensionsSpec()).build()
|
||||
),
|
||||
finisher,
|
||||
cnf,
|
||||
optimize
|
||||
);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() throws Exception
|
||||
{
|
||||
BaseFilterTest.tearDown(ColumnComparisonFilterTest.class.getName());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneSingleValuedStringColumn()
|
||||
{
|
||||
assertFilterMatches(EDF("dim3 == ''"), ImmutableList.of("0"));
|
||||
assertFilterMatches(EDF("dim3 == '1'"), ImmutableList.of("3", "4", "6"));
|
||||
assertFilterMatches(EDF("dim3 == 'a'"), ImmutableList.of("7"));
|
||||
assertFilterMatches(EDF("dim3 == 1"), ImmutableList.of("3", "4", "6"));
|
||||
assertFilterMatches(EDF("dim3 == 1.0"), ImmutableList.of("3", "4", "6"));
|
||||
assertFilterMatches(EDF("dim3 == 1.234"), ImmutableList.of("9"));
|
||||
assertFilterMatches(EDF("dim3 < '2'"), ImmutableList.of("0", "1", "3", "4", "6", "9"));
|
||||
assertFilterMatches(EDF("dim3 < 2"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
|
||||
assertFilterMatches(EDF("dim3 < 2.0"), ImmutableList.of("0", "3", "4", "6", "7", "9"));
|
||||
assertFilterMatches(EDF("like(dim3, '1%')"), ImmutableList.of("1", "3", "4", "6", "9"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneMultiValuedStringColumn()
|
||||
{
|
||||
// Expressions currently treat multi-valued arrays as nulls.
|
||||
// This test is just documenting the current behavior, not necessarily saying it makes sense.
|
||||
|
||||
assertFilterMatches(EDF("dim4 == ''"), ImmutableList.of("0", "1", "2", "4", "5", "6", "7", "8"));
|
||||
assertFilterMatches(EDF("dim4 == '1'"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("dim4 == '3'"), ImmutableList.of("3"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneLongColumn()
|
||||
{
|
||||
assertFilterMatches(EDF("dim1 == ''"), ImmutableList.of("0"));
|
||||
assertFilterMatches(EDF("dim1 == '1'"), ImmutableList.of("1"));
|
||||
assertFilterMatches(EDF("dim1 == 2"), ImmutableList.of("2"));
|
||||
assertFilterMatches(EDF("dim1 < '2'"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(EDF("dim1 < 2"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(EDF("dim1 < 2.0"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(EDF("like(dim1, '1%')"), ImmutableList.of("1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testOneFloatColumn()
|
||||
{
|
||||
assertFilterMatches(EDF("dim2 == ''"), ImmutableList.of("0"));
|
||||
assertFilterMatches(EDF("dim2 == '1'"), ImmutableList.of("1"));
|
||||
assertFilterMatches(EDF("dim2 == 2"), ImmutableList.of("2"));
|
||||
assertFilterMatches(EDF("dim2 < '2'"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(EDF("dim2 < 2"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(EDF("dim2 < 2.0"), ImmutableList.of("0", "1"));
|
||||
assertFilterMatches(EDF("like(dim2, '1%')"), ImmutableList.of("1"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testConstantExpression()
|
||||
{
|
||||
assertFilterMatches(EDF("1 + 1"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
|
||||
assertFilterMatches(EDF("0 + 0"), ImmutableList.of());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCompareColumns()
|
||||
{
|
||||
// String vs string
|
||||
assertFilterMatches(EDF("dim0 == dim3"), ImmutableList.of("2", "5", "8"));
|
||||
|
||||
// String vs long
|
||||
assertFilterMatches(EDF("dim1 == dim3"), ImmutableList.of("0", "2", "5", "8"));
|
||||
|
||||
// String vs float
|
||||
assertFilterMatches(EDF("dim2 == dim3"), ImmutableList.of("0", "2", "5", "8"));
|
||||
|
||||
// String vs. multi-value string
|
||||
// Expressions currently treat multi-valued arrays as nulls.
|
||||
// This test is just documenting the current behavior, not necessarily saying it makes sense.
|
||||
assertFilterMatches(EDF("dim0 == dim4"), ImmutableList.of("3"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingColumn()
|
||||
{
|
||||
assertFilterMatches(EDF("missing == ''"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
|
||||
assertFilterMatches(EDF("missing == '1'"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("missing == 2"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("missing < '2'"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
|
||||
assertFilterMatches(EDF("missing < 2"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
|
||||
assertFilterMatches(EDF("missing < 2.0"), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
|
||||
assertFilterMatches(EDF("missing > '2'"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("missing > 2"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("missing > 2.0"), ImmutableList.of());
|
||||
assertFilterMatches(EDF("like(missing, '1%')"), ImmutableList.of());
|
||||
}
|
||||
|
||||
private static ExpressionDimFilter EDF(final String expression)
|
||||
{
|
||||
return new ExpressionDimFilter(expression, TestExprMacroTable.INSTANCE);
|
||||
}
|
||||
}
|
|
@ -97,6 +97,12 @@ public class ExtractionDimFilterTest
|
|||
return vals == null ? null : new ArrayIndexed<String>(vals, String.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasMultipleValues(final String dimension)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue