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:
Gian Merlino 2016-11-30 16:10:05 -08:00 committed by GitHub
parent e4465e63bd
commit 477e0cab7c
24 changed files with 483 additions and 429 deletions

View File

@ -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) {

View File

@ -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
}
};
}
}

View File

@ -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.

View File

@ -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()));
}
}
}

View File

@ -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;
}
};
}
}
}
}

View File

@ -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);
/**

View File

@ -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 {

View File

@ -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);
}

View File

@ -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()

View File

@ -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();

View File

@ -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));
}
}

View File

@ -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")
);
}
}

View File

@ -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));
}
}

View File

@ -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);
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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);
}
};
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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));
}
}