mirror of https://github.com/apache/druid.git
Filter fixes and tests (#3724)
* More robust Filter tests. All Filter tests now exercise the CNF and post-filtering features. * Fixes to RowBasedValueMatcherFactory and to bound filters. - Change Comparables to Strings in ValueMatcher related code. - Break out RowBasedValueMatcherFactory, fix a variety of issues around nulls, and add tests. - Fix bound filters on long columns with non-numeric bounds, and add tests.
This commit is contained in:
parent
e4465e63bd
commit
477e0cab7c
|
@ -219,7 +219,7 @@ public class FilteredAggregatorFactory extends AggregatorFactory
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final Comparable value)
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final String value)
|
||||
{
|
||||
if (getTypeForDimension(dimension) == ValueType.LONG) {
|
||||
return Filters.getLongValueMatcher(
|
||||
|
@ -232,8 +232,8 @@ public class FilteredAggregatorFactory extends AggregatorFactory
|
|||
new DefaultDimensionSpec(dimension, dimension)
|
||||
);
|
||||
|
||||
// Compare "value" as a String.
|
||||
final String valueString = value == null ? null : Strings.emptyToNull(value.toString());
|
||||
// Compare "value" as null if it's empty.
|
||||
final String valueString = Strings.emptyToNull(value);
|
||||
|
||||
// Missing columns match a null or empty string value, and don't match anything else.
|
||||
if (selector == null) {
|
||||
|
|
|
@ -220,7 +220,8 @@ public class BoundDimFilter implements DimFilter
|
|||
range = isLowerStrict() ? Range.greaterThan(getLower()) : Range.atLeast(getLower());
|
||||
} else {
|
||||
range = Range.range(getLower(), isLowerStrict() ? BoundType.OPEN : BoundType.CLOSED,
|
||||
getUpper(), isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED);
|
||||
getUpper(), isUpperStrict() ? BoundType.OPEN : BoundType.CLOSED
|
||||
);
|
||||
}
|
||||
retSet.add(range);
|
||||
return retSet;
|
||||
|
@ -317,6 +318,7 @@ public class BoundDimFilter implements DimFilter
|
|||
{
|
||||
private final Object initLock = new Object();
|
||||
private volatile boolean longsInitialized = false;
|
||||
private volatile boolean matchesAnything = true;
|
||||
private volatile boolean hasLowerLongBoundVolatile;
|
||||
private volatile boolean hasUpperLongBoundVolatile;
|
||||
private volatile long lowerLongBoundVolatile;
|
||||
|
@ -327,6 +329,17 @@ public class BoundDimFilter implements DimFilter
|
|||
{
|
||||
initLongData();
|
||||
|
||||
if (!matchesAnything) {
|
||||
return new DruidLongPredicate()
|
||||
{
|
||||
@Override
|
||||
public boolean applyLong(long input)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
return new DruidLongPredicate()
|
||||
{
|
||||
private final boolean hasLowerLongBound = hasLowerLongBoundVolatile;
|
||||
|
@ -369,16 +382,26 @@ public class BoundDimFilter implements DimFilter
|
|||
return;
|
||||
}
|
||||
|
||||
Long lowerLong = GuavaUtils.tryParseLong(lower);
|
||||
if (hasLowerBound() && lowerLong != null) {
|
||||
if (hasLowerBound()) {
|
||||
final Long lowerLong = GuavaUtils.tryParseLong(lower);
|
||||
if (lowerLong == null) {
|
||||
matchesAnything = false;
|
||||
return;
|
||||
}
|
||||
|
||||
hasLowerLongBoundVolatile = true;
|
||||
lowerLongBoundVolatile = lowerLong;
|
||||
} else {
|
||||
hasLowerLongBoundVolatile = false;
|
||||
}
|
||||
|
||||
Long upperLong = GuavaUtils.tryParseLong(upper);
|
||||
if (hasUpperBound() && upperLong != null) {
|
||||
if (hasUpperBound()) {
|
||||
Long upperLong = GuavaUtils.tryParseLong(upper);
|
||||
if (upperLong == null) {
|
||||
matchesAnything = false;
|
||||
return;
|
||||
}
|
||||
|
||||
hasUpperLongBoundVolatile = true;
|
||||
upperLongBoundVolatile = upperLong;
|
||||
} else {
|
||||
|
@ -390,6 +413,4 @@ public class BoundDimFilter implements DimFilter
|
|||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ package io.druid.query.filter;
|
|||
public interface ValueMatcherFactory
|
||||
{
|
||||
/**
|
||||
* Create a ValueMatcher that compares row values to the provided value.
|
||||
* Create a ValueMatcher that compares row values to the provided string.
|
||||
*
|
||||
* An implementation of this method should be able to handle dimensions of various types.
|
||||
*
|
||||
|
@ -44,8 +44,7 @@ public interface ValueMatcherFactory
|
|||
*
|
||||
* @return An object that matches row values on the provided value.
|
||||
*/
|
||||
public ValueMatcher makeValueMatcher(String dimension, Comparable value);
|
||||
|
||||
public ValueMatcher makeValueMatcher(String dimension, String value);
|
||||
|
||||
/**
|
||||
* Create a ValueMatcher that applies a predicate to row values.
|
||||
|
|
|
@ -0,0 +1,164 @@
|
|||
/*
|
||||
* 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.groupby;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import io.druid.common.guava.GuavaUtils;
|
||||
import io.druid.data.input.Row;
|
||||
import io.druid.query.filter.DruidLongPredicate;
|
||||
import io.druid.query.filter.DruidPredicateFactory;
|
||||
import io.druid.query.filter.ValueMatcher;
|
||||
import io.druid.query.filter.ValueMatcherFactory;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.filter.BooleanValueMatcher;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
|
||||
public class RowBasedValueMatcherFactory implements ValueMatcherFactory
|
||||
{
|
||||
private Row row;
|
||||
|
||||
public void setRow(Row row)
|
||||
{
|
||||
this.row = row;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final String value)
|
||||
{
|
||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||
if (value == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
|
||||
final Long longValue = GuavaUtils.tryParseLong(value);
|
||||
if (longValue == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
|
||||
return new ValueMatcher()
|
||||
{
|
||||
// store the primitive, so we don't unbox for every comparison
|
||||
final long unboxedLong = longValue;
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return row.getTimestampFromEpoch() == unboxedLong;
|
||||
}
|
||||
};
|
||||
} else {
|
||||
final String valueOrNull = Strings.emptyToNull(value);
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return doesMatch(row.getRaw(dimension), valueOrNull);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
// There is no easy way to determine column types from a Row, so this generates all possible predicates and then
|
||||
// uses instanceof checks to determine which one to apply to each row.
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return predicate.applyLong(row.getTimestampFromEpoch());
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
final Predicate<String> stringPredicate = predicateFactory.makeStringPredicate();
|
||||
final DruidLongPredicate longPredicate = predicateFactory.makeLongPredicate();
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return doesMatch(row.getRaw(dimension), stringPredicate, longPredicate);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
// Precondition: value must be run through Strings.emptyToNull
|
||||
private boolean doesMatch(final Object raw, final String value)
|
||||
{
|
||||
if (raw == null) {
|
||||
return value == null;
|
||||
} else if (raw instanceof List) {
|
||||
final List theList = (List) raw;
|
||||
if (theList.isEmpty()) {
|
||||
// null should match empty rows in multi-value columns
|
||||
return value == null;
|
||||
} else {
|
||||
for (Object o : theList) {
|
||||
if (doesMatch(o, value)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
} else {
|
||||
return Objects.equals(Strings.emptyToNull(raw.toString()), value);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean doesMatch(
|
||||
final Object raw,
|
||||
final Predicate<String> stringPredicate,
|
||||
final DruidLongPredicate longPredicate
|
||||
)
|
||||
{
|
||||
if (raw == null) {
|
||||
return stringPredicate.apply(null);
|
||||
} else if (raw instanceof List) {
|
||||
final List theList = (List) raw;
|
||||
if (theList.isEmpty()) {
|
||||
return stringPredicate.apply(null);
|
||||
} else {
|
||||
for (Object o : theList) {
|
||||
if (doesMatch(o, stringPredicate, longPredicate)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
}
|
||||
} else if (raw instanceof Long || raw instanceof Integer) {
|
||||
return longPredicate.applyLong(((Number) raw).longValue());
|
||||
} else {
|
||||
return stringPredicate.apply(Strings.emptyToNull(raw.toString()));
|
||||
}
|
||||
}
|
||||
}
|
|
@ -37,15 +37,12 @@ import io.druid.query.QueryContextKeys;
|
|||
import io.druid.query.QueryInterruptedException;
|
||||
import io.druid.query.ResourceLimitExceededException;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.filter.DruidLongPredicate;
|
||||
import io.druid.query.filter.DruidPredicateFactory;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.ValueMatcher;
|
||||
import io.druid.query.filter.ValueMatcherFactory;
|
||||
import io.druid.query.groupby.GroupByQuery;
|
||||
import io.druid.query.groupby.GroupByQueryConfig;
|
||||
import io.druid.query.groupby.RowBasedValueMatcherFactory;
|
||||
import io.druid.query.groupby.epinephelinae.RowBasedGrouperHelper.RowBasedKey;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.filter.BooleanValueMatcher;
|
||||
import io.druid.segment.filter.Filters;
|
||||
import org.joda.time.DateTime;
|
||||
|
@ -196,76 +193,4 @@ public class GroupByRowProcessor
|
|||
}
|
||||
);
|
||||
}
|
||||
|
||||
private static class RowBasedValueMatcherFactory implements ValueMatcherFactory
|
||||
{
|
||||
private Row row;
|
||||
|
||||
public void setRow(Row row)
|
||||
{
|
||||
this.row = row;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final Comparable value)
|
||||
{
|
||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return value.equals(row.getTimestampFromEpoch());
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return row.getDimension(dimension).contains(value);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
|
||||
// There is no easy way to determine the dimension value type from the map based row, so this defaults all
|
||||
// dimensions (except time) to string, and provide the string value matcher. This has some performance impact
|
||||
// on filtering, but should provide the same result. It should be changed to support dimension types when better
|
||||
// type hinting is implemented
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final DruidPredicateFactory predicateFactory)
|
||||
{
|
||||
if (dimension.equals(Column.TIME_COLUMN_NAME)) {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
final DruidLongPredicate predicate = predicateFactory.makeLongPredicate();
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return predicate.applyLong(row.getTimestampFromEpoch());
|
||||
}
|
||||
};
|
||||
} else {
|
||||
return new ValueMatcher()
|
||||
{
|
||||
final Predicate<String> predicate = predicateFactory.makeStringPredicate();
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
List<String> values = row.getDimension(dimension);
|
||||
for (String value : values) {
|
||||
if (predicate.apply(value)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -326,7 +326,7 @@ public interface DimensionIndexer<EncodedType extends Comparable<EncodedType>, E
|
|||
* @param dimIndex the array index of this indexer's dimension within the TimeAndDims key
|
||||
* @return A ValueMatcher that matches a dimension value array from a TimeAndDims key against "matchValue"
|
||||
*/
|
||||
public ValueMatcher makeIndexingValueMatcher(Comparable matchValue, IncrementalIndexStorageAdapter.EntryHolder holder, int dimIndex);
|
||||
public ValueMatcher makeIndexingValueMatcher(String matchValue, IncrementalIndexStorageAdapter.EntryHolder holder, int dimIndex);
|
||||
|
||||
|
||||
/**
|
||||
|
|
|
@ -1025,14 +1025,6 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
}
|
||||
|
||||
private static boolean isComparableNullOrEmpty(final Comparable value)
|
||||
{
|
||||
if (value instanceof String) {
|
||||
return Strings.isNullOrEmpty((String) value);
|
||||
}
|
||||
return value == null;
|
||||
}
|
||||
|
||||
private static class CursorOffsetHolderValueMatcherFactory implements ValueMatcherFactory
|
||||
{
|
||||
private final ColumnSelector index;
|
||||
|
@ -1048,7 +1040,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(String dimension, final Comparable value)
|
||||
public ValueMatcher makeValueMatcher(String dimension, final String value)
|
||||
{
|
||||
if (getTypeForDimension(dimension) == ValueType.LONG) {
|
||||
return Filters.getLongValueMatcher(
|
||||
|
@ -1062,9 +1054,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
|||
);
|
||||
|
||||
// if matching against null, rows with size 0 should also match
|
||||
final boolean matchNull = isComparableNullOrEmpty(value);
|
||||
final boolean matchNull = Strings.isNullOrEmpty(value);
|
||||
|
||||
final int id = selector.lookupId((String) value);
|
||||
final int id = selector.lookupId(value);
|
||||
if (id < 0) {
|
||||
return new BooleanValueMatcher(false);
|
||||
} else {
|
||||
|
|
|
@ -536,14 +536,13 @@ public class StringDimensionIndexer implements DimensionIndexer<Integer, int[],
|
|||
|
||||
@Override
|
||||
public ValueMatcher makeIndexingValueMatcher(
|
||||
final Comparable matchValue,
|
||||
final String matchValue,
|
||||
final IncrementalIndexStorageAdapter.EntryHolder holder,
|
||||
final int dimIndex
|
||||
)
|
||||
{
|
||||
final String value = STRING_TRANSFORMER.apply(matchValue);
|
||||
final int encodedVal = getEncodedValue(value, false);
|
||||
final boolean matchOnNull = Strings.isNullOrEmpty(value);
|
||||
final int encodedVal = getEncodedValue(matchValue, false);
|
||||
final boolean matchOnNull = Strings.isNullOrEmpty(matchValue);
|
||||
if (encodedVal < 0 && !matchOnNull) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.segment.filter;
|
|||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
|
@ -175,14 +176,14 @@ public class Filters
|
|||
|
||||
public static ValueMatcher getLongValueMatcher(
|
||||
final LongColumnSelector longSelector,
|
||||
Comparable value
|
||||
final String value
|
||||
)
|
||||
{
|
||||
if (value == null) {
|
||||
if (Strings.isNullOrEmpty(value)) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
|
||||
final Long longValue = GuavaUtils.tryParseLong(value.toString());
|
||||
final Long longValue = GuavaUtils.tryParseLong(value);
|
||||
if (longValue == null) {
|
||||
return new BooleanValueMatcher(false);
|
||||
}
|
||||
|
@ -190,7 +191,7 @@ public class Filters
|
|||
return new ValueMatcher()
|
||||
{
|
||||
// store the primitive, so we don't unbox for every comparison
|
||||
final long unboxedLong = longValue.longValue();
|
||||
final long unboxedLong = longValue;
|
||||
|
||||
@Override
|
||||
public boolean matches()
|
||||
|
|
|
@ -600,14 +600,6 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
);
|
||||
}
|
||||
|
||||
private boolean isComparableNullOrEmpty(final Comparable value)
|
||||
{
|
||||
if (value instanceof String) {
|
||||
return Strings.isNullOrEmpty((String) value);
|
||||
}
|
||||
return value == null;
|
||||
}
|
||||
|
||||
private ValueMatcher makeFilterMatcher(final Filter filter, final Cursor cursor, final EntryHolder holder)
|
||||
{
|
||||
return filter == null
|
||||
|
@ -656,7 +648,7 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(String dimension, final Comparable originalValue)
|
||||
public ValueMatcher makeValueMatcher(String dimension, final String originalValue)
|
||||
{
|
||||
IncrementalIndex.DimensionDesc dimensionDesc = index.getDimension(dimension);
|
||||
if (dimensionDesc == null) {
|
||||
|
@ -668,10 +660,10 @@ public class IncrementalIndexStorageAdapter implements StorageAdapter
|
|||
case LONG:
|
||||
return Filters.getLongValueMatcher(cursor.makeLongColumnSelector(dimension), originalValue);
|
||||
default:
|
||||
return new BooleanValueMatcher(isComparableNullOrEmpty(originalValue));
|
||||
return new BooleanValueMatcher(Strings.isNullOrEmpty(originalValue));
|
||||
}
|
||||
} else {
|
||||
return new BooleanValueMatcher(isComparableNullOrEmpty(originalValue));
|
||||
return new BooleanValueMatcher(Strings.isNullOrEmpty(originalValue));
|
||||
}
|
||||
} else {
|
||||
final DimensionIndexer indexer = dimensionDesc.getIndexer();
|
||||
|
|
|
@ -37,7 +37,6 @@ import io.druid.segment.IndexBuilder;
|
|||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -71,10 +70,11 @@ public class AndFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -176,13 +176,4 @@ public class AndFilterTest extends BaseFilterTest
|
|||
ImmutableList.<String>of("0", "1", "2", "3", "4", "5")
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.google.common.base.Preconditions;
|
|||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.granularity.QueryGranularities;
|
||||
|
@ -34,8 +35,12 @@ import io.druid.query.aggregation.Aggregator;
|
|||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.FilteredAggregatorFactory;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.ValueMatcher;
|
||||
import io.druid.query.filter.ValueMatcherFactory;
|
||||
import io.druid.query.groupby.RowBasedValueMatcherFactory;
|
||||
import io.druid.segment.Cursor;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
|
@ -52,6 +57,7 @@ import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
|||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexStorageAdapter;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
@ -76,6 +82,7 @@ public abstract class BaseFilterTest
|
|||
protected final Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher;
|
||||
protected StorageAdapter adapter;
|
||||
protected Closeable closeable;
|
||||
protected boolean cnf;
|
||||
protected boolean optimize;
|
||||
protected final String testName;
|
||||
|
||||
|
@ -85,19 +92,20 @@ public abstract class BaseFilterTest
|
|||
// Each thread gets its own map.
|
||||
protected static ThreadLocal<Map<String, Map<String, Pair<StorageAdapter, Closeable>>>> adapterCache =
|
||||
new ThreadLocal<Map<String, Map<String, Pair<StorageAdapter, Closeable>>>>()
|
||||
{
|
||||
@Override
|
||||
protected Map<String, Map<String, Pair<StorageAdapter, Closeable>>> initialValue()
|
||||
{
|
||||
return new HashMap<>();
|
||||
}
|
||||
};
|
||||
{
|
||||
@Override
|
||||
protected Map<String, Map<String, Pair<StorageAdapter, Closeable>>> initialValue()
|
||||
{
|
||||
return new HashMap<>();
|
||||
}
|
||||
};
|
||||
|
||||
public BaseFilterTest(
|
||||
String testName,
|
||||
List<InputRow> rows,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
|
@ -105,6 +113,7 @@ public abstract class BaseFilterTest
|
|||
this.rows = rows;
|
||||
this.indexBuilder = indexBuilder;
|
||||
this.finisher = finisher;
|
||||
this.cnf = cnf;
|
||||
this.optimize = optimize;
|
||||
}
|
||||
|
||||
|
@ -121,7 +130,7 @@ public abstract class BaseFilterTest
|
|||
Pair<StorageAdapter, Closeable> pair = adaptersForClass.get(testName);
|
||||
if (pair == null) {
|
||||
pair = finisher.apply(
|
||||
indexBuilder.tmpDir(temporaryFolder.newFolder()).add(rows)
|
||||
indexBuilder.tmpDir(temporaryFolder.newFolder()).rows(rows)
|
||||
);
|
||||
adaptersForClass.put(testName, pair);
|
||||
}
|
||||
|
@ -227,24 +236,26 @@ public abstract class BaseFilterTest
|
|||
for (Map.Entry<String, BitmapSerdeFactory> bitmapSerdeFactoryEntry : bitmapSerdeFactories.entrySet()) {
|
||||
for (Map.Entry<String, IndexMerger> indexMergerEntry : indexMergers.entrySet()) {
|
||||
for (Map.Entry<String, Function<IndexBuilder, Pair<StorageAdapter, Closeable>>> finisherEntry : finishers.entrySet()) {
|
||||
for (boolean optimize : ImmutableList.of(false, true)) {
|
||||
final String testName = String.format(
|
||||
"bitmaps[%s], indexMerger[%s], finisher[%s], optimize[%s]",
|
||||
bitmapSerdeFactoryEntry.getKey(),
|
||||
indexMergerEntry.getKey(),
|
||||
finisherEntry.getKey(),
|
||||
optimize
|
||||
);
|
||||
final IndexBuilder indexBuilder = IndexBuilder.create()
|
||||
.indexSpec(new IndexSpec(
|
||||
bitmapSerdeFactoryEntry.getValue(),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
))
|
||||
.indexMerger(indexMergerEntry.getValue());
|
||||
for (boolean cnf : ImmutableList.of(false, true)) {
|
||||
for (boolean optimize : ImmutableList.of(false, true)) {
|
||||
final String testName = String.format(
|
||||
"bitmaps[%s], indexMerger[%s], finisher[%s], optimize[%s]",
|
||||
bitmapSerdeFactoryEntry.getKey(),
|
||||
indexMergerEntry.getKey(),
|
||||
finisherEntry.getKey(),
|
||||
optimize
|
||||
);
|
||||
final IndexBuilder indexBuilder = IndexBuilder.create()
|
||||
.indexSpec(new IndexSpec(
|
||||
bitmapSerdeFactoryEntry.getValue(),
|
||||
null,
|
||||
null,
|
||||
null
|
||||
))
|
||||
.indexMerger(indexMergerEntry.getValue());
|
||||
|
||||
constructors.add(new Object[]{testName, indexBuilder, finisherEntry.getValue(), optimize});
|
||||
constructors.add(new Object[]{testName, indexBuilder, finisherEntry.getValue(), cnf, optimize});
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -253,7 +264,18 @@ public abstract class BaseFilterTest
|
|||
return constructors;
|
||||
}
|
||||
|
||||
protected DimFilter maybeOptimize(final DimFilter dimFilter)
|
||||
private Filter makeFilter(final DimFilter dimFilter)
|
||||
{
|
||||
if (dimFilter == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
final DimFilter maybeOptimized = optimize ? dimFilter.optimize() : dimFilter;
|
||||
final Filter filter = maybeOptimized.toFilter();
|
||||
return cnf ? Filters.convertToCNF(filter) : filter;
|
||||
}
|
||||
|
||||
private DimFilter maybeOptimize(final DimFilter dimFilter)
|
||||
{
|
||||
if (dimFilter == null) {
|
||||
return null;
|
||||
|
@ -261,7 +283,7 @@ public abstract class BaseFilterTest
|
|||
return optimize ? dimFilter.optimize() : dimFilter;
|
||||
}
|
||||
|
||||
protected Sequence<Cursor> makeCursorSequence(final Filter filter)
|
||||
private Sequence<Cursor> makeCursorSequence(final Filter filter)
|
||||
{
|
||||
final Sequence<Cursor> cursors = adapter.makeCursors(
|
||||
filter,
|
||||
|
@ -276,9 +298,9 @@ public abstract class BaseFilterTest
|
|||
/**
|
||||
* Selects elements from "selectColumn" from rows matching a filter. selectColumn must be a single valued dimension.
|
||||
*/
|
||||
protected List<String> selectColumnValuesMatchingFilter(final DimFilter filter, final String selectColumn)
|
||||
private List<String> selectColumnValuesMatchingFilter(final DimFilter filter, final String selectColumn)
|
||||
{
|
||||
final Sequence<Cursor> cursors = makeCursorSequence(Filters.toFilter(maybeOptimize(filter)));
|
||||
final Sequence<Cursor> cursors = makeCursorSequence(makeFilter(filter));
|
||||
Sequence<List<String>> seq = Sequences.map(
|
||||
cursors,
|
||||
new Function<Cursor, List<String>>()
|
||||
|
@ -306,9 +328,9 @@ public abstract class BaseFilterTest
|
|||
return Sequences.toList(seq, new ArrayList<List<String>>()).get(0);
|
||||
}
|
||||
|
||||
protected long selectCountUsingFilteredAggregator(final DimFilter filter)
|
||||
private long selectCountUsingFilteredAggregator(final DimFilter filter)
|
||||
{
|
||||
final Sequence<Cursor> cursors = makeCursorSequence(Filters.toFilter(maybeOptimize(filter)));
|
||||
final Sequence<Cursor> cursors = makeCursorSequence(makeFilter(filter));
|
||||
Sequence<Aggregator> aggSeq = Sequences.map(
|
||||
cursors,
|
||||
new Function<Cursor, Aggregator>()
|
||||
|
@ -331,4 +353,103 @@ public abstract class BaseFilterTest
|
|||
);
|
||||
return Sequences.toList(aggSeq, new ArrayList<Aggregator>()).get(0).getLong();
|
||||
}
|
||||
|
||||
private List<String> selectColumnValuesMatchingFilterUsingPostFiltering(
|
||||
final DimFilter filter,
|
||||
final String selectColumn
|
||||
)
|
||||
{
|
||||
final Filter theFilter = makeFilter(filter);
|
||||
final Filter postFilteringFilter = new Filter()
|
||||
{
|
||||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
||||
{
|
||||
return theFilter.makeMatcher(factory);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
};
|
||||
|
||||
final Sequence<Cursor> cursors = makeCursorSequence(postFilteringFilter);
|
||||
Sequence<List<String>> seq = Sequences.map(
|
||||
cursors,
|
||||
new Function<Cursor, List<String>>()
|
||||
{
|
||||
@Override
|
||||
public List<String> apply(Cursor input)
|
||||
{
|
||||
final DimensionSelector selector = input.makeDimensionSelector(
|
||||
new DefaultDimensionSpec(selectColumn, selectColumn)
|
||||
);
|
||||
|
||||
final List<String> values = Lists.newArrayList();
|
||||
|
||||
while (!input.isDone()) {
|
||||
IndexedInts row = selector.getRow();
|
||||
Preconditions.checkState(row.size() == 1);
|
||||
values.add(selector.lookupName(row.get(0)));
|
||||
input.advance();
|
||||
}
|
||||
|
||||
return values;
|
||||
}
|
||||
}
|
||||
);
|
||||
return Sequences.toList(seq, new ArrayList<List<String>>()).get(0);
|
||||
}
|
||||
|
||||
private List<String> selectColumnValuesMatchingFilterUsingRowBasedValueMatcherFactory(
|
||||
final DimFilter filter,
|
||||
final String selectColumn
|
||||
)
|
||||
{
|
||||
final RowBasedValueMatcherFactory matcherFactory = new RowBasedValueMatcherFactory();
|
||||
final ValueMatcher matcher = makeFilter(filter).makeMatcher(matcherFactory);
|
||||
final List<String> values = Lists.newArrayList();
|
||||
for (InputRow row : rows) {
|
||||
matcherFactory.setRow(row);
|
||||
if (matcher.matches()) {
|
||||
values.add((String) row.getRaw(selectColumn));
|
||||
}
|
||||
}
|
||||
return values;
|
||||
}
|
||||
|
||||
protected void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(
|
||||
"Cursor: " + filter.toString(),
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilter(filter, "dim0")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"Cursor with postFiltering: " + filter.toString(),
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingPostFiltering(filter, "dim0")
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"Filtered aggregator: " + filter.toString(),
|
||||
expectedRows.size(),
|
||||
selectCountUsingFilteredAggregator(filter)
|
||||
);
|
||||
Assert.assertEquals(
|
||||
"RowBasedValueMatcherFactory: " + filter.toString(),
|
||||
expectedRows,
|
||||
selectColumnValuesMatchingFilterUsingRowBasedValueMatcherFactory(filter, "dim0")
|
||||
);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,13 +33,11 @@ import io.druid.js.JavaScriptConfig;
|
|||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.ordering.StringComparators;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -75,10 +73,11 @@ public class BoundFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -464,13 +463,4 @@ public class BoundFilterTest extends BaseFilterTest
|
|||
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7")
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,12 +20,10 @@
|
|||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
|
@ -33,10 +31,7 @@ import io.druid.data.input.impl.MapInputRowParser;
|
|||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||
import io.druid.query.filter.AndDimFilter;
|
||||
|
@ -47,11 +42,8 @@ import io.druid.query.filter.DruidPredicateFactory;
|
|||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.OrDimFilter;
|
||||
import io.druid.query.filter.SelectorDimFilter;
|
||||
import io.druid.segment.Cursor;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
|
@ -60,7 +52,6 @@ import org.junit.runner.RunWith;
|
|||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -191,10 +182,11 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -558,12 +550,6 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
ImmutableList.of("4", "6")
|
||||
);
|
||||
|
||||
assertFilterMatchesCNF(
|
||||
dimFilter1,
|
||||
ImmutableList.of("4", "6")
|
||||
);
|
||||
|
||||
|
||||
DimFilter dimFilter2 = new OrDimFilter(Arrays.<DimFilter>asList(
|
||||
new SelectorDimFilter("dim0", "2", null),
|
||||
new SelectorDimFilter("dim0", "3", null),
|
||||
|
@ -579,12 +565,6 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
ImmutableList.of("2", "3", "7")
|
||||
);
|
||||
|
||||
assertFilterMatchesCNF(
|
||||
dimFilter2,
|
||||
ImmutableList.of("2", "3", "7")
|
||||
);
|
||||
|
||||
|
||||
DimFilter dimFilter3 = new OrDimFilter(Arrays.<DimFilter>asList(
|
||||
dimFilter1,
|
||||
dimFilter2,
|
||||
|
@ -595,21 +575,12 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
))
|
||||
);
|
||||
|
||||
Filter filter3 = dimFilter3.toFilter();
|
||||
Filter filter3CNF = Filters.convertToCNF(dimFilter3.toFilter());
|
||||
|
||||
assertFilterMatches(
|
||||
dimFilter3,
|
||||
ImmutableList.of("2", "3", "4", "6", "7", "9")
|
||||
);
|
||||
|
||||
assertFilterMatchesCNF(
|
||||
dimFilter3,
|
||||
ImmutableList.of("2", "3", "4", "6", "7", "9")
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testDistributeOrCNFExtractionFn()
|
||||
{
|
||||
|
@ -633,12 +604,6 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
ImmutableList.of("4", "6")
|
||||
);
|
||||
|
||||
assertFilterMatchesCNF(
|
||||
dimFilter1,
|
||||
ImmutableList.of("4", "6")
|
||||
);
|
||||
|
||||
|
||||
DimFilter dimFilter2 = new OrDimFilter(Arrays.<DimFilter>asList(
|
||||
new SelectorDimFilter("dim0", "super-2", JS_EXTRACTION_FN),
|
||||
new SelectorDimFilter("dim0", "super-3", JS_EXTRACTION_FN),
|
||||
|
@ -654,12 +619,6 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
ImmutableList.of("2", "3", "7")
|
||||
);
|
||||
|
||||
assertFilterMatchesCNF(
|
||||
dimFilter2,
|
||||
ImmutableList.of("2", "3", "7")
|
||||
);
|
||||
|
||||
|
||||
DimFilter dimFilter3 = new OrDimFilter(Arrays.<DimFilter>asList(
|
||||
dimFilter1,
|
||||
dimFilter2,
|
||||
|
@ -677,62 +636,5 @@ public class FilterPartitionTest extends BaseFilterTest
|
|||
dimFilter3,
|
||||
ImmutableList.of("2", "3", "4", "6", "7", "9")
|
||||
);
|
||||
|
||||
assertFilterMatchesCNF(
|
||||
dimFilter3,
|
||||
ImmutableList.of("2", "3", "4", "6", "7", "9")
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
|
||||
private void assertFilterMatchesCNF(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilterCNF(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
|
||||
protected List<String> selectColumnValuesMatchingFilterCNF(final DimFilter dimFilter, final String selectColumn)
|
||||
{
|
||||
final Filter filter = Filters.convertToCNF(maybeOptimize(dimFilter).toFilter());
|
||||
|
||||
final Sequence<Cursor> cursors = makeCursorSequence(filter);
|
||||
Sequence<List<String>> seq = Sequences.map(
|
||||
cursors,
|
||||
new Function<Cursor, List<String>>()
|
||||
{
|
||||
@Override
|
||||
public List<String> apply(Cursor input)
|
||||
{
|
||||
final DimensionSelector selector = input.makeDimensionSelector(
|
||||
new DefaultDimensionSpec(selectColumn, selectColumn)
|
||||
);
|
||||
|
||||
final List<String> values = Lists.newArrayList();
|
||||
|
||||
while (!input.isDone()) {
|
||||
IndexedInts row = selector.getRow();
|
||||
Preconditions.checkState(row.size() == 1);
|
||||
values.add(selector.lookupName(row.get(0)));
|
||||
input.advance();
|
||||
}
|
||||
|
||||
return values;
|
||||
}
|
||||
}
|
||||
);
|
||||
return Sequences.toList(seq, new ArrayList<List<String>>()).get(0);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -42,7 +42,6 @@ import io.druid.segment.IndexBuilder;
|
|||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -76,10 +75,11 @@ public class InFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -297,13 +297,4 @@ public class InFilterTest extends BaseFilterTest
|
|||
{
|
||||
return new InDimFilter(dim, Lists.asList(value, values), fn);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,7 +33,6 @@ import io.druid.query.aggregation.AggregatorFactory;
|
|||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleMaxAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.InDimFilter;
|
||||
import io.druid.query.filter.SelectorDimFilter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
|
@ -41,7 +40,6 @@ import io.druid.segment.StorageAdapter;
|
|||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -88,10 +86,11 @@ public class InvalidFilteringTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, overrideIndexBuilderSchema(indexBuilder), finisher, optimize);
|
||||
super(testName, ROWS, overrideIndexBuilderSchema(indexBuilder), finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
private static IndexBuilder overrideIndexBuilderSchema(IndexBuilder indexBuilder)
|
||||
|
@ -156,13 +155,4 @@ public class InvalidFilteringTest extends BaseFilterTest
|
|||
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ import io.druid.java.util.common.Pair;
|
|||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.extraction.MapLookupExtractor;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.JavaScriptDimFilter;
|
||||
import io.druid.query.lookup.LookupExtractionFn;
|
||||
import io.druid.query.lookup.LookupExtractor;
|
||||
|
@ -40,7 +39,6 @@ import io.druid.segment.IndexBuilder;
|
|||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -78,10 +76,11 @@ public class JavaScriptFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -189,13 +188,4 @@ public class JavaScriptFilterTest extends BaseFilterTest
|
|||
JavaScriptConfig.getDefault()
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,13 +30,11 @@ 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.extraction.SubstringDimExtractionFn;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.LikeDimFilter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -70,10 +68,11 @@ public class LikeFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -207,13 +206,4 @@ public class LikeFilterTest extends BaseFilterTest
|
|||
ImmutableList.of("2", "4")
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,8 @@ import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
|||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.extraction.MapLookupExtractor;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
|
@ -48,6 +50,7 @@ import io.druid.query.ordering.StringComparators;
|
|||
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
||||
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.Assert;
|
||||
|
@ -67,7 +70,7 @@ import java.util.concurrent.TimeUnit;
|
|||
@RunWith(Parameterized.class)
|
||||
public class LongFilteringTest extends BaseFilterTest
|
||||
{
|
||||
private static final String COUNT_COLUMN = "count";
|
||||
private static final String LONG_COLUMN = "lng";
|
||||
private static final String TIMESTAMP_COLUMN = "ts";
|
||||
private static int EXECUTOR_NUM_THREADS = 16;
|
||||
private static int EXECUTOR_NUM_TASKS = 2000;
|
||||
|
@ -83,30 +86,38 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
)
|
||||
);
|
||||
|
||||
private static final InputRow row0 = PARSER.parse(ImmutableMap.<String, Object>of("ts", 1L, "dim0", "1", "dim1", "", "dim2", ImmutableList.of("a", "b")));
|
||||
private static final InputRow row1 = PARSER.parse(ImmutableMap.<String, Object>of("ts", 2L, "dim0", "2", "dim1", "10", "dim2", ImmutableList.of()));
|
||||
private static final InputRow row2 = PARSER.parse(ImmutableMap.<String, Object>of("ts", 3L, "dim0", "3", "dim1", "2", "dim2", ImmutableList.of("")));
|
||||
private static final InputRow row3 = PARSER.parse(ImmutableMap.<String, Object>of("ts", 4L, "dim0", "4", "dim1", "1", "dim2", ImmutableList.of("a")));
|
||||
private static final InputRow row4 = PARSER.parse(ImmutableMap.<String, Object>of("ts", 5L, "dim0", "5", "dim1", "def", "dim2", ImmutableList.of("c")));
|
||||
private static final InputRow row5 = PARSER.parse(ImmutableMap.<String, Object>of("ts", 6L, "dim0", "6", "dim1", "abc"));
|
||||
|
||||
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||
row0,
|
||||
row1, row1,
|
||||
row2, row2, row2,
|
||||
row3, row3, row3, row3,
|
||||
row4, row4, row4, row4, row4,
|
||||
row5, row5, row5, row5, row5, row5
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("ts", 1L, "dim0", "1", "lng", 1L, "dim1", "", "dim2", ImmutableList.of("a", "b"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("ts", 2L, "dim0", "2", "lng", 2L, "dim1", "10", "dim2", ImmutableList.of())),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("ts", 3L, "dim0", "3", "lng", 3L, "dim1", "2", "dim2", ImmutableList.of(""))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("ts", 4L, "dim0", "4", "lng", 4L, "dim1", "1", "dim2", ImmutableList.of("a"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("ts", 5L, "dim0", "5", "lng", 5L, "dim1", "def", "dim2", ImmutableList.of("c"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("ts", 6L, "dim0", "6", "lng", 6L, "dim1", "abc"))
|
||||
);
|
||||
|
||||
public LongFilteringTest(
|
||||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(
|
||||
testName,
|
||||
ROWS,
|
||||
indexBuilder.schema(
|
||||
new IncrementalIndexSchema.Builder()
|
||||
.withMetrics(
|
||||
new AggregatorFactory[]{
|
||||
new LongSumAggregatorFactory(LONG_COLUMN, LONG_COLUMN)
|
||||
}
|
||||
).build()
|
||||
),
|
||||
finisher,
|
||||
cnf,
|
||||
optimize
|
||||
);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -116,30 +127,30 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testTimeFilterAsLong()
|
||||
public void testLongColumnFiltering()
|
||||
{
|
||||
assertFilterMatches(
|
||||
new SelectorDimFilter(COUNT_COLUMN, "0", null),
|
||||
new SelectorDimFilter(LONG_COLUMN, "0", null),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new SelectorDimFilter(COUNT_COLUMN, "3", null),
|
||||
new SelectorDimFilter(LONG_COLUMN, "3", null),
|
||||
ImmutableList.<String>of("3")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(COUNT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
|
||||
new BoundDimFilter(LONG_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
|
||||
ImmutableList.<String>of("2", "3", "4", "5")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(COUNT_COLUMN, "1", "4", true, true, null, null, StringComparators.NUMERIC),
|
||||
new BoundDimFilter(LONG_COLUMN, "1", "4", true, true, null, null, StringComparators.NUMERIC),
|
||||
ImmutableList.<String>of("2", "3")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new InDimFilter(COUNT_COLUMN, Arrays.asList("2", "4", "8"), null),
|
||||
new InDimFilter(LONG_COLUMN, Arrays.asList("2", "4", "8"), null),
|
||||
ImmutableList.<String>of("2", "4")
|
||||
);
|
||||
|
||||
|
@ -149,27 +160,71 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
infilterValues.add(String.valueOf(i * 2));
|
||||
}
|
||||
assertFilterMatches(
|
||||
new InDimFilter(COUNT_COLUMN, infilterValues, null),
|
||||
new InDimFilter(LONG_COLUMN, infilterValues, null),
|
||||
ImmutableList.<String>of("2", "4", "6")
|
||||
);
|
||||
|
||||
String jsFn = "function(x) { return(x === 3 || x === 5) }";
|
||||
assertFilterMatches(
|
||||
new JavaScriptDimFilter(COUNT_COLUMN, jsFn, null, JavaScriptConfig.getDefault()),
|
||||
new JavaScriptDimFilter(LONG_COLUMN, jsFn, null, JavaScriptConfig.getDefault()),
|
||||
ImmutableList.<String>of("3", "5")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new RegexDimFilter(COUNT_COLUMN, "4", null),
|
||||
new RegexDimFilter(LONG_COLUMN, "4", null),
|
||||
ImmutableList.<String>of("4")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new SearchQueryDimFilter(COUNT_COLUMN, new ContainsSearchQuerySpec("2", true), null),
|
||||
new SearchQueryDimFilter(LONG_COLUMN, new ContainsSearchQuerySpec("2", true), null),
|
||||
ImmutableList.<String>of("2")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLongColumnFilteringWithNonNumbers()
|
||||
{
|
||||
assertFilterMatches(
|
||||
new SelectorDimFilter(LONG_COLUMN, "", null),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new SelectorDimFilter(LONG_COLUMN, null, null),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new SelectorDimFilter(LONG_COLUMN, "abc", null),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(LONG_COLUMN, "a", "b", false, false, null, null, StringComparators.NUMERIC),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(LONG_COLUMN, " ", "4", false, false, null, null, StringComparators.NUMERIC),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(LONG_COLUMN, " ", "4", false, false, null, null, StringComparators.LEXICOGRAPHIC),
|
||||
ImmutableList.<String>of("1", "2", "3", "4")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(LONG_COLUMN, " ", "A", false, false, null, null, StringComparators.NUMERIC),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(LONG_COLUMN, " ", "A", false, false, null, null, StringComparators.LEXICOGRAPHIC),
|
||||
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLongFilterWithExtractionFn()
|
||||
{
|
||||
|
@ -184,25 +239,25 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
LookupExtractionFn exfn = new LookupExtractionFn(mapExtractor, false, "UNKNOWN", false, true);
|
||||
|
||||
assertFilterMatches(
|
||||
new SelectorDimFilter(COUNT_COLUMN, "Monday", exfn),
|
||||
new SelectorDimFilter(LONG_COLUMN, "Monday", exfn),
|
||||
ImmutableList.<String>of("1")
|
||||
);
|
||||
assertFilterMatches(
|
||||
new SelectorDimFilter(COUNT_COLUMN, "Notaday", exfn),
|
||||
new SelectorDimFilter(LONG_COLUMN, "Notaday", exfn),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(COUNT_COLUMN, "Fridax", "Fridaz", false, false, null, exfn, StringComparators.ALPHANUMERIC),
|
||||
new BoundDimFilter(LONG_COLUMN, "Fridax", "Fridaz", false, false, null, exfn, StringComparators.ALPHANUMERIC),
|
||||
ImmutableList.<String>of("5")
|
||||
);
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(COUNT_COLUMN, "Friday", "Friday", true, true, null, exfn, StringComparators.ALPHANUMERIC),
|
||||
new BoundDimFilter(LONG_COLUMN, "Friday", "Friday", true, true, null, exfn, StringComparators.ALPHANUMERIC),
|
||||
ImmutableList.<String>of()
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new InDimFilter(COUNT_COLUMN, Arrays.asList("Caturday", "Saturday", "Tuesday"), exfn),
|
||||
new InDimFilter(LONG_COLUMN, Arrays.asList("Caturday", "Saturday", "Tuesday"), exfn),
|
||||
ImmutableList.<String>of("2", "6")
|
||||
);
|
||||
|
||||
|
@ -213,23 +268,23 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
"Hello", "World", "1", "2", "3", "4", "5", "6", "7"
|
||||
);
|
||||
assertFilterMatches(
|
||||
new InDimFilter(COUNT_COLUMN, bigList, exfn),
|
||||
new InDimFilter(LONG_COLUMN, bigList, exfn),
|
||||
ImmutableList.<String>of("2", "6")
|
||||
);
|
||||
|
||||
String jsFn = "function(x) { return(x === 'Wednesday' || x === 'Thursday') }";
|
||||
assertFilterMatches(
|
||||
new JavaScriptDimFilter(COUNT_COLUMN, jsFn, exfn, JavaScriptConfig.getDefault()),
|
||||
new JavaScriptDimFilter(LONG_COLUMN, jsFn, exfn, JavaScriptConfig.getDefault()),
|
||||
ImmutableList.<String>of("3", "4")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new RegexDimFilter(COUNT_COLUMN, ".*day", exfn),
|
||||
new RegexDimFilter(LONG_COLUMN, ".*day", exfn),
|
||||
ImmutableList.<String>of("1", "2", "3", "4", "5", "6")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new SearchQueryDimFilter(COUNT_COLUMN, new ContainsSearchQuerySpec("s", true), exfn),
|
||||
new SearchQueryDimFilter(LONG_COLUMN, new ContainsSearchQuerySpec("s", true), exfn),
|
||||
ImmutableList.<String>of("2", "3", "4")
|
||||
);
|
||||
}
|
||||
|
@ -238,12 +293,12 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
public void testMultithreaded()
|
||||
{
|
||||
assertFilterMatchesMultithreaded(
|
||||
new SelectorDimFilter(COUNT_COLUMN, "3", null),
|
||||
new SelectorDimFilter(LONG_COLUMN, "3", null),
|
||||
ImmutableList.<String>of("3")
|
||||
);
|
||||
|
||||
assertFilterMatchesMultithreaded(
|
||||
new InDimFilter(COUNT_COLUMN, Arrays.asList("2", "4", "8"), null),
|
||||
new InDimFilter(LONG_COLUMN, Arrays.asList("2", "4", "8"), null),
|
||||
ImmutableList.<String>of("2", "4")
|
||||
);
|
||||
|
||||
|
@ -253,25 +308,16 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
infilterValues.add(String.valueOf(i * 2));
|
||||
}
|
||||
assertFilterMatchesMultithreaded(
|
||||
new InDimFilter(COUNT_COLUMN, infilterValues, null),
|
||||
new InDimFilter(LONG_COLUMN, infilterValues, null),
|
||||
ImmutableList.<String>of("2", "4", "6")
|
||||
);
|
||||
|
||||
assertFilterMatches(
|
||||
new BoundDimFilter(COUNT_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
|
||||
new BoundDimFilter(LONG_COLUMN, "2", "5", false, false, null, null, StringComparators.NUMERIC),
|
||||
ImmutableList.<String>of("2", "3", "4", "5")
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
|
||||
private void assertFilterMatchesMultithreaded(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
|
@ -290,8 +336,7 @@ public class LongFilteringTest extends BaseFilterTest
|
|||
@Override
|
||||
public void run()
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
assertFilterMatches(filter, expectedRows);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -29,14 +29,12 @@ import io.druid.data.input.impl.MapInputRowParser;
|
|||
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.filter.DimFilter;
|
||||
import io.druid.query.filter.NotDimFilter;
|
||||
import io.druid.query.filter.SelectorDimFilter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -70,10 +68,11 @@ public class NotFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -102,13 +101,4 @@ public class NotFilterTest extends BaseFilterTest
|
|||
ImmutableList.of("0", "2", "3", "4", "5")
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,13 +32,11 @@ import io.druid.java.util.common.Pair;
|
|||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.RegexDimFilter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -76,10 +74,11 @@ public class RegexFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -155,13 +154,4 @@ public class RegexFilterTest extends BaseFilterTest
|
|||
assertFilterMatches(new RegexDimFilter("dim4", ".*ANYMORE", changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||
assertFilterMatches(new RegexDimFilter("dim4", "a.*", changeNullFn), ImmutableList.<String>of());
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,6 @@ import io.druid.java.util.common.Pair;
|
|||
import io.druid.js.JavaScriptConfig;
|
||||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.extraction.JavaScriptExtractionFn;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.SearchQueryDimFilter;
|
||||
import io.druid.query.search.search.ContainsSearchQuerySpec;
|
||||
import io.druid.query.search.search.SearchQuerySpec;
|
||||
|
@ -40,7 +39,6 @@ import io.druid.segment.IndexBuilder;
|
|||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -78,10 +76,11 @@ public class SearchQueryFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -164,13 +163,4 @@ public class SearchQueryFilterTest extends BaseFilterTest
|
|||
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("ANYMORE"), changeNullFn), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||
assertFilterMatches(new SearchQueryDimFilter("dim4", specForValue("a"), changeNullFn), ImmutableList.<String>of());
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,7 +30,6 @@ 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.extraction.MapLookupExtractor;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.ExtractionDimFilter;
|
||||
import io.druid.query.filter.InDimFilter;
|
||||
import io.druid.query.filter.SelectorDimFilter;
|
||||
|
@ -79,10 +78,11 @@ public class SelectorFilterTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -235,13 +235,4 @@ public class SelectorFilterTest extends BaseFilterTest
|
|||
assertFilterMatches(new ExtractionDimFilter("dim0", "5", lookupFn2, null), ImmutableList.of("2", "5"));
|
||||
assertFilterMatches(new ExtractionDimFilter("dim0", null, lookupFn3, null), ImmutableList.of("0", "1", "2", "3", "4", "5"));
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,6 @@ import io.druid.query.extraction.JavaScriptExtractionFn;
|
|||
import io.druid.query.extraction.MapLookupExtractor;
|
||||
import io.druid.query.extraction.TimeFormatExtractionFn;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.InDimFilter;
|
||||
import io.druid.query.filter.IntervalDimFilter;
|
||||
import io.druid.query.filter.JavaScriptDimFilter;
|
||||
|
@ -53,7 +52,6 @@ import org.joda.time.DateTime;
|
|||
import org.joda.time.DateTimeZone;
|
||||
import org.joda.time.Interval;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
@ -94,10 +92,11 @@ public class TimeFilteringTest extends BaseFilterTest
|
|||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
|
||||
boolean cnf,
|
||||
boolean optimize
|
||||
)
|
||||
{
|
||||
super(testName, ROWS, indexBuilder, finisher, optimize);
|
||||
super(testName, ROWS, indexBuilder, finisher, cnf, optimize);
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
|
@ -341,13 +340,4 @@ public class TimeFilteringTest extends BaseFilterTest
|
|||
ImmutableList.<String>of("1", "2", "3", "4")
|
||||
);
|
||||
}
|
||||
|
||||
private void assertFilterMatches(
|
||||
final DimFilter filter,
|
||||
final List<String> expectedRows
|
||||
)
|
||||
{
|
||||
Assert.assertEquals(filter.toString(), expectedRows, selectColumnValuesMatchingFilter(filter, "dim0"));
|
||||
Assert.assertEquals(filter.toString(), expectedRows.size(), selectCountUsingFilteredAggregator(filter));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue