mirror of https://github.com/apache/druid.git
BoundFilter optimizations, and related interface changes.
BoundFilter: - For lexicographic bounds, use bitmapIndex.getIndex to find the start and end points, then union all bitmaps between those points. - For alphanumeric bounds, iterate through dimValues, and union all bitmaps for values matching the predicate. - Change behavior for nulls: it used to be that the BoundFilter would never match nulls, now it matches nulls if "" is allowed by the lower limit and not excluded by the upper limit. Interface changes: - BitmapIndex: add `int getIndex(value)` to make it possible to get the index for a value without retrieving the bitmap. - BitmapIndex: remove `ImmutableBitmap getBitmap(value)`, change callers to `getBitmap(getIndex(value))`. - BitmapIndexSelector: allow retrieving the underlying BitmapIndex through getBitmapIndex. - Clarified contract of indexOf in Indexed, GenericIndexed. Also added tests for SelectorFilter, NotFilter, and BoundFilter.
This commit is contained in:
parent
02d0c704b1
commit
2970b49adc
|
@ -0,0 +1,290 @@
|
|||
/*
|
||||
* 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.benchmark;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.FluentIterable;
|
||||
import com.google.common.io.BaseEncoding;
|
||||
import com.google.common.primitives.Ints;
|
||||
import com.metamx.collections.bitmap.BitmapFactory;
|
||||
import com.metamx.collections.bitmap.ConciseBitmapFactory;
|
||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||
import com.metamx.collections.bitmap.MutableBitmap;
|
||||
import com.metamx.collections.bitmap.RoaringBitmapFactory;
|
||||
import com.metamx.collections.spatial.ImmutableRTree;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.data.BitmapSerdeFactory;
|
||||
import io.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||
import io.druid.segment.data.GenericIndexed;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import io.druid.segment.filter.BoundFilter;
|
||||
import io.druid.segment.serde.BitmapIndexColumnPartSupplier;
|
||||
import it.uniroma3.mat.extendedset.intset.ConciseSetUtils;
|
||||
import org.openjdk.jmh.annotations.Benchmark;
|
||||
import org.openjdk.jmh.annotations.BenchmarkMode;
|
||||
import org.openjdk.jmh.annotations.Fork;
|
||||
import org.openjdk.jmh.annotations.Measurement;
|
||||
import org.openjdk.jmh.annotations.Mode;
|
||||
import org.openjdk.jmh.annotations.OutputTimeUnit;
|
||||
import org.openjdk.jmh.annotations.Param;
|
||||
import org.openjdk.jmh.annotations.Scope;
|
||||
import org.openjdk.jmh.annotations.Setup;
|
||||
import org.openjdk.jmh.annotations.State;
|
||||
import org.openjdk.jmh.annotations.Warmup;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@State(Scope.Benchmark)
|
||||
@Fork(value = 1)
|
||||
@Warmup(iterations = 10)
|
||||
@Measurement(iterations = 10)
|
||||
public class BoundFilterBenchmark
|
||||
{
|
||||
private static final int START_INT = 1_000_000_000;
|
||||
private static final int END_INT = ConciseSetUtils.MAX_ALLOWED_INTEGER;
|
||||
|
||||
private static final BoundFilter NOTHING_LEXICOGRAPHIC = new BoundFilter(
|
||||
new BoundDimFilter(
|
||||
"foo",
|
||||
String.valueOf(START_INT),
|
||||
String.valueOf(START_INT),
|
||||
true,
|
||||
false,
|
||||
false
|
||||
)
|
||||
);
|
||||
|
||||
private static final BoundFilter HALF_LEXICOGRAPHIC = new BoundFilter(
|
||||
new BoundDimFilter(
|
||||
"foo",
|
||||
String.valueOf(START_INT + (END_INT - START_INT) / 2),
|
||||
String.valueOf(END_INT),
|
||||
false,
|
||||
false,
|
||||
false
|
||||
)
|
||||
);
|
||||
|
||||
private static final BoundFilter EVERYTHING_LEXICOGRAPHIC = new BoundFilter(
|
||||
new BoundDimFilter(
|
||||
"foo",
|
||||
String.valueOf(START_INT),
|
||||
String.valueOf(END_INT),
|
||||
false,
|
||||
false,
|
||||
false
|
||||
)
|
||||
);
|
||||
|
||||
private static final BoundFilter NOTHING_ALPHANUMERIC = new BoundFilter(
|
||||
new BoundDimFilter(
|
||||
"foo",
|
||||
String.valueOf(START_INT),
|
||||
String.valueOf(START_INT),
|
||||
true,
|
||||
false,
|
||||
true
|
||||
)
|
||||
);
|
||||
|
||||
private static final BoundFilter HALF_ALPHANUMERIC = new BoundFilter(
|
||||
new BoundDimFilter(
|
||||
"foo",
|
||||
String.valueOf(START_INT + (END_INT - START_INT) / 2),
|
||||
String.valueOf(END_INT),
|
||||
false,
|
||||
false,
|
||||
true
|
||||
)
|
||||
);
|
||||
|
||||
private static final BoundFilter EVERYTHING_ALPHANUMERIC = new BoundFilter(
|
||||
new BoundDimFilter(
|
||||
"foo",
|
||||
String.valueOf(START_INT),
|
||||
String.valueOf(END_INT),
|
||||
false,
|
||||
false,
|
||||
true
|
||||
)
|
||||
);
|
||||
|
||||
// cardinality, the dictionary will contain evenly spaced integers
|
||||
@Param({"1000", "100000", "1000000"})
|
||||
int cardinality;
|
||||
|
||||
int step;
|
||||
|
||||
// selector will contain a cardinality number of bitmaps; each one contains a single int: 0
|
||||
BitmapIndexSelector selector;
|
||||
|
||||
@Setup
|
||||
public void setup() throws IOException
|
||||
{
|
||||
step = (END_INT - START_INT) / cardinality;
|
||||
final BitmapFactory bitmapFactory = new RoaringBitmapFactory();
|
||||
final BitmapSerdeFactory serdeFactory = new RoaringBitmapSerdeFactory();
|
||||
final List<Integer> ints = generateInts();
|
||||
final GenericIndexed<String> dictionary = GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(
|
||||
new Function<Integer, String>()
|
||||
{
|
||||
@Override
|
||||
public String apply(Integer i)
|
||||
{
|
||||
return i.toString();
|
||||
}
|
||||
}
|
||||
),
|
||||
GenericIndexed.STRING_STRATEGY
|
||||
);
|
||||
final BitmapIndex bitmapIndex = new BitmapIndexColumnPartSupplier(
|
||||
bitmapFactory,
|
||||
GenericIndexed.fromIterable(
|
||||
FluentIterable.from(ints)
|
||||
.transform(
|
||||
new Function<Integer, ImmutableBitmap>()
|
||||
{
|
||||
@Override
|
||||
public ImmutableBitmap apply(Integer i)
|
||||
{
|
||||
final MutableBitmap mutableBitmap = bitmapFactory.makeEmptyMutableBitmap();
|
||||
mutableBitmap.add((i - START_INT) / step);
|
||||
return bitmapFactory.makeImmutableBitmap(mutableBitmap);
|
||||
}
|
||||
}
|
||||
),
|
||||
serdeFactory.getObjectStrategy()
|
||||
),
|
||||
dictionary
|
||||
).get();
|
||||
selector = new BitmapIndexSelector()
|
||||
{
|
||||
@Override
|
||||
public Indexed<String> getDimensionValues(String dimension)
|
||||
{
|
||||
return dictionary;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getNumRows()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapFactory getBitmapFactory()
|
||||
{
|
||||
return bitmapFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, String value)
|
||||
{
|
||||
return bitmapIndex.getBitmap(bitmapIndex.getIndex(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapIndex getBitmapIndex(String dimension)
|
||||
{
|
||||
return bitmapIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void matchNothingLexicographic()
|
||||
{
|
||||
final ImmutableBitmap bitmapIndex = NOTHING_LEXICOGRAPHIC.getBitmapIndex(selector);
|
||||
Preconditions.checkState(bitmapIndex.size() == 0);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void matchHalfLexicographic()
|
||||
{
|
||||
final ImmutableBitmap bitmapIndex = HALF_LEXICOGRAPHIC.getBitmapIndex(selector);
|
||||
Preconditions.checkState(bitmapIndex.size() > 0 && bitmapIndex.size() < cardinality);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void matchEverythingLexicographic()
|
||||
{
|
||||
final ImmutableBitmap bitmapIndex = EVERYTHING_LEXICOGRAPHIC.getBitmapIndex(selector);
|
||||
Preconditions.checkState(bitmapIndex.size() == cardinality);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void matchNothingAlphaNumeric()
|
||||
{
|
||||
final ImmutableBitmap bitmapIndex = NOTHING_ALPHANUMERIC.getBitmapIndex(selector);
|
||||
Preconditions.checkState(bitmapIndex.size() == 0);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void matchHalfAlphaNumeric()
|
||||
{
|
||||
final ImmutableBitmap bitmapIndex = HALF_ALPHANUMERIC.getBitmapIndex(selector);
|
||||
Preconditions.checkState(bitmapIndex.size() > 0 && bitmapIndex.size() < cardinality);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void matchEverythingAlphaNumeric()
|
||||
{
|
||||
final ImmutableBitmap bitmapIndex = EVERYTHING_ALPHANUMERIC.getBitmapIndex(selector);
|
||||
Preconditions.checkState(bitmapIndex.size() == cardinality);
|
||||
}
|
||||
|
||||
private List<Integer> generateInts()
|
||||
{
|
||||
final List<Integer> ints = new ArrayList<>(cardinality);
|
||||
|
||||
for (int i = 0; i < cardinality; i++) {
|
||||
ints.add(START_INT + step * i);
|
||||
}
|
||||
|
||||
return ints;
|
||||
}
|
||||
}
|
|
@ -22,6 +22,7 @@ package io.druid.query.filter;
|
|||
import com.metamx.collections.bitmap.BitmapFactory;
|
||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||
import com.metamx.collections.spatial.ImmutableRTree;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.data.Indexed;
|
||||
|
||||
/**
|
||||
|
@ -31,6 +32,7 @@ public interface BitmapIndexSelector
|
|||
public Indexed<String> getDimensionValues(String dimension);
|
||||
public int getNumRows();
|
||||
public BitmapFactory getBitmapFactory();
|
||||
public BitmapIndex getBitmapIndex(String dimension);
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, String value);
|
||||
public ImmutableRTree getSpatialIndex(String dimension);
|
||||
}
|
||||
|
|
|
@ -91,6 +91,16 @@ public class BoundDimFilter implements DimFilter
|
|||
return alphaNumeric;
|
||||
}
|
||||
|
||||
public boolean hasLowerBound()
|
||||
{
|
||||
return lower != null;
|
||||
}
|
||||
|
||||
public boolean hasUpperBound()
|
||||
{
|
||||
return upper != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public byte[] getCacheKey()
|
||||
{
|
||||
|
|
|
@ -206,7 +206,7 @@ public class SegmentAnalyzer
|
|||
for (int i = 0; i < cardinality; ++i) {
|
||||
String value = bitmapIndex.getValue(i);
|
||||
if (value != null) {
|
||||
size += StringUtils.toUtf8(value).length * bitmapIndex.getBitmap(value).size();
|
||||
size += StringUtils.toUtf8(value).length * bitmapIndex.getBitmap(bitmapIndex.getIndex(value)).size();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
|
|||
import com.metamx.collections.spatial.ImmutableRTree;
|
||||
import com.metamx.common.guava.CloseQuietly;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.DictionaryEncodedColumn;
|
||||
import io.druid.segment.column.GenericColumn;
|
||||
|
@ -110,6 +111,17 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
return bitmapFactory;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapIndex getBitmapIndex(String dimension)
|
||||
{
|
||||
final Column column = index.getColumn(dimension);
|
||||
if (column != null && column.getCapabilities().hasBitmapIndexes()) {
|
||||
return column.getBitmapIndex();
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(String dimension, String value)
|
||||
{
|
||||
|
@ -126,7 +138,8 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
|
|||
return bitmapFactory.makeEmptyImmutableBitmap();
|
||||
}
|
||||
|
||||
return column.getBitmapIndex().getBitmap(value);
|
||||
final BitmapIndex bitmapIndex = column.getBitmapIndex();
|
||||
return bitmapIndex.getBitmap(bitmapIndex.getIndex(value));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -309,7 +309,7 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
|
|||
return EmptyIndexedInts.EMPTY_INDEXED_INTS;
|
||||
}
|
||||
|
||||
return new BitmapCompressedIndexedInts(bitmaps.getBitmap(value));
|
||||
return new BitmapCompressedIndexedInts(bitmaps.getBitmap(bitmaps.getIndex(value)));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -34,7 +34,14 @@ public interface BitmapIndex
|
|||
|
||||
public BitmapFactory getBitmapFactory();
|
||||
|
||||
public ImmutableBitmap getBitmap(String value);
|
||||
/**
|
||||
* Returns the index of "value" in this BitmapIndex, or (-(insertion point) - 1) if the value is not
|
||||
* present, in the manner of Arrays.binarySearch.
|
||||
*
|
||||
* @param value value to search for
|
||||
* @return index of value, or negative number equal to (-(insertion point) - 1).
|
||||
*/
|
||||
public int getIndex(String value);
|
||||
|
||||
public ImmutableBitmap getBitmap(int idx);
|
||||
}
|
||||
|
|
|
@ -127,6 +127,14 @@ public class GenericIndexed<T> implements Indexed<T>
|
|||
return bufferIndexed.get(index);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the index of "value" in this GenericIndexed object, or (-(insertion point) - 1) if the value is not
|
||||
* present, in the manner of Arrays.binarySearch. This strengthens the contract of Indexed, which only guarantees
|
||||
* that values-not-found will return some negative number.
|
||||
*
|
||||
* @param value value to search for
|
||||
* @return index of value, or negative number equal to (-(insertion point) - 1).
|
||||
*/
|
||||
@Override
|
||||
public int indexOf(T value)
|
||||
{
|
||||
|
|
|
@ -22,7 +22,19 @@ package io.druid.segment.data;
|
|||
public interface Indexed<T> extends Iterable<T>
|
||||
{
|
||||
Class<? extends T> getClazz();
|
||||
|
||||
int size();
|
||||
|
||||
T get(int index);
|
||||
|
||||
/**
|
||||
* Returns the index of "value" in this Indexed object, or a negative number if the value is not present.
|
||||
* The negative number is not guaranteed to be any particular number. Subclasses may tighten this contract
|
||||
* (GenericIndexed does this).
|
||||
*
|
||||
* @param value value to search for
|
||||
*
|
||||
* @return index of value, or a negative number
|
||||
*/
|
||||
int indexOf(T value);
|
||||
}
|
||||
|
|
|
@ -20,67 +20,194 @@
|
|||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Predicate;
|
||||
import com.metamx.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.query.topn.AlphaNumericTopNMetricSpec;
|
||||
import io.druid.query.topn.LexicographicTopNMetricSpec;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.ValueMatcher;
|
||||
import io.druid.query.filter.ValueMatcherFactory;
|
||||
import io.druid.query.ordering.StringComparators;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.data.Indexed;
|
||||
|
||||
import java.util.Comparator;
|
||||
import java.util.Iterator;
|
||||
|
||||
public class BoundFilter extends DimensionPredicateFilter
|
||||
public class BoundFilter implements Filter
|
||||
{
|
||||
private final BoundDimFilter boundDimFilter;
|
||||
private final Comparator<String> comparator;
|
||||
|
||||
public BoundFilter(final BoundDimFilter boundDimFilter)
|
||||
{
|
||||
super(
|
||||
boundDimFilter.getDimension(), new Predicate<String>()
|
||||
{
|
||||
private volatile Predicate<String> predicate;
|
||||
this.boundDimFilter = boundDimFilter;
|
||||
this.comparator = boundDimFilter.isAlphaNumeric()
|
||||
? StringComparators.ALPHANUMERIC
|
||||
: StringComparators.LEXICOGRAPHIC;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean apply(String input)
|
||||
{
|
||||
return function().apply(input);
|
||||
}
|
||||
@Override
|
||||
public ImmutableBitmap getBitmapIndex(final BitmapIndexSelector selector)
|
||||
{
|
||||
final BitmapIndex bitmapIndex = selector.getBitmapIndex(boundDimFilter.getDimension());
|
||||
|
||||
private Predicate<String> function()
|
||||
if (bitmapIndex == null) {
|
||||
if (doesMatch(null)) {
|
||||
return selector.getBitmapFactory()
|
||||
.complement(selector.getBitmapFactory().makeEmptyImmutableBitmap(), selector.getNumRows());
|
||||
} else {
|
||||
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||
}
|
||||
}
|
||||
|
||||
if (boundDimFilter.isAlphaNumeric()) {
|
||||
// inspect all values
|
||||
|
||||
// will be non-null because bitmapIndex was non-null
|
||||
final Indexed<String> dimValues = selector.getDimensionValues(boundDimFilter.getDimension());
|
||||
|
||||
return selector.getBitmapFactory().union(
|
||||
new Iterable<ImmutableBitmap>()
|
||||
{
|
||||
if (predicate == null) {
|
||||
final Comparator<String> comparator;
|
||||
if (boundDimFilter.isAlphaNumeric()) {
|
||||
comparator = new AlphaNumericTopNMetricSpec(null).getComparator(null, null);
|
||||
} else {
|
||||
comparator = new LexicographicTopNMetricSpec(null).getComparator(null, null);
|
||||
}
|
||||
predicate = new Predicate<String>()
|
||||
@Override
|
||||
public Iterator<ImmutableBitmap> iterator()
|
||||
{
|
||||
return new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
int currIndex = 0;
|
||||
|
||||
@Override
|
||||
public boolean apply(String input)
|
||||
public boolean hasNext()
|
||||
{
|
||||
if (input == null) {
|
||||
return false;
|
||||
return currIndex < bitmapIndex.getCardinality();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableBitmap next()
|
||||
{
|
||||
while (currIndex < bitmapIndex.getCardinality() && !doesMatch(dimValues.get(currIndex))) {
|
||||
currIndex++;
|
||||
}
|
||||
int lowerComparing = 1;
|
||||
int upperComparing = 1;
|
||||
if (boundDimFilter.getLower() != null) {
|
||||
lowerComparing = comparator.compare(input, boundDimFilter.getLower());
|
||||
|
||||
if (currIndex == bitmapIndex.getCardinality()) {
|
||||
return bitmapIndex.getBitmapFactory().makeEmptyImmutableBitmap();
|
||||
}
|
||||
if (boundDimFilter.getUpper() != null) {
|
||||
upperComparing = comparator.compare(boundDimFilter.getUpper(), input);
|
||||
}
|
||||
if (boundDimFilter.isLowerStrict() && boundDimFilter.isUpperStrict()) {
|
||||
return ((lowerComparing > 0)) && (upperComparing > 0);
|
||||
} else if (boundDimFilter.isLowerStrict()) {
|
||||
return (lowerComparing > 0) && (upperComparing >= 0);
|
||||
} else if (boundDimFilter.isUpperStrict()) {
|
||||
return (lowerComparing >= 0) && (upperComparing > 0);
|
||||
}
|
||||
return (lowerComparing >= 0) && (upperComparing >= 0);
|
||||
|
||||
return bitmapIndex.getBitmap(currIndex++);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
return predicate;
|
||||
}
|
||||
);
|
||||
} else {
|
||||
// search for start, end indexes in the bitmaps; then include all bitmaps between those points
|
||||
|
||||
final int startIndex; // inclusive
|
||||
final int endIndex; // exclusive
|
||||
|
||||
if (!boundDimFilter.hasLowerBound()) {
|
||||
startIndex = 0;
|
||||
} else {
|
||||
final int found = bitmapIndex.getIndex(boundDimFilter.getLower());
|
||||
if (found >= 0) {
|
||||
startIndex = boundDimFilter.isLowerStrict() ? found + 1 : found;
|
||||
} else {
|
||||
startIndex = -(found + 1);
|
||||
}
|
||||
}
|
||||
|
||||
if (!boundDimFilter.hasUpperBound()) {
|
||||
endIndex = bitmapIndex.getCardinality();
|
||||
} else {
|
||||
final int found = bitmapIndex.getIndex(boundDimFilter.getUpper());
|
||||
if (found >= 0) {
|
||||
endIndex = boundDimFilter.isUpperStrict() ? found : found + 1;
|
||||
} else {
|
||||
endIndex = -(found + 1);
|
||||
}
|
||||
}
|
||||
|
||||
return selector.getBitmapFactory().union(
|
||||
new Iterable<ImmutableBitmap>()
|
||||
{
|
||||
@Override
|
||||
public Iterator<ImmutableBitmap> iterator()
|
||||
{
|
||||
return new Iterator<ImmutableBitmap>()
|
||||
{
|
||||
int currIndex = startIndex;
|
||||
|
||||
@Override
|
||||
public boolean hasNext()
|
||||
{
|
||||
return currIndex < endIndex;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableBitmap next()
|
||||
{
|
||||
return bitmapIndex.getBitmap(currIndex++);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void remove()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
|
||||
{
|
||||
return factory.makeValueMatcher(
|
||||
boundDimFilter.getDimension(),
|
||||
new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(String input)
|
||||
{
|
||||
return doesMatch(input);
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private boolean doesMatch(String input)
|
||||
{
|
||||
if (input == null) {
|
||||
return (!boundDimFilter.hasLowerBound()
|
||||
|| (boundDimFilter.getLower().isEmpty() && !boundDimFilter.isLowerStrict())) // lower bound allows null
|
||||
&& (!boundDimFilter.hasUpperBound()
|
||||
|| !boundDimFilter.getUpper().isEmpty()
|
||||
|| !boundDimFilter.isUpperStrict()) // upper bound allows null
|
||||
;
|
||||
}
|
||||
int lowerComparing = 1;
|
||||
int upperComparing = 1;
|
||||
if (boundDimFilter.hasLowerBound()) {
|
||||
lowerComparing = comparator.compare(input, boundDimFilter.getLower());
|
||||
}
|
||||
if (boundDimFilter.hasUpperBound()) {
|
||||
upperComparing = comparator.compare(boundDimFilter.getUpper(), input);
|
||||
}
|
||||
if (boundDimFilter.isLowerStrict() && boundDimFilter.isUpperStrict()) {
|
||||
return ((lowerComparing > 0)) && (upperComparing > 0);
|
||||
} else if (boundDimFilter.isLowerStrict()) {
|
||||
return (lowerComparing > 0) && (upperComparing >= 0);
|
||||
} else if (boundDimFilter.isUpperStrict()) {
|
||||
return (lowerComparing >= 0) && (upperComparing > 0);
|
||||
}
|
||||
return (lowerComparing >= 0) && (upperComparing >= 0);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -74,11 +74,10 @@ public class BitmapIndexColumnPartSupplier implements Supplier<BitmapIndex>
|
|||
}
|
||||
|
||||
@Override
|
||||
public ImmutableBitmap getBitmap(String value)
|
||||
public int getIndex(String value)
|
||||
{
|
||||
final int index = dictionary.indexOf(value);
|
||||
|
||||
return getBitmap(index);
|
||||
// GenericIndexed.indexOf satisfies contract needed by BitmapIndex.indexOf
|
||||
return dictionary.indexOf(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -36,9 +36,9 @@ import java.io.IOException;
|
|||
import java.util.Arrays;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class BoundDimFilterTests
|
||||
public class BoundDimFilterTest
|
||||
{
|
||||
public BoundDimFilterTests(BoundDimFilter boundDimFilter) {this.boundDimFilter = boundDimFilter;}
|
||||
public BoundDimFilterTest(BoundDimFilter boundDimFilter) {this.boundDimFilter = boundDimFilter;}
|
||||
|
||||
private final BoundDimFilter boundDimFilter;
|
||||
|
|
@ -2184,7 +2184,7 @@ public class TimeseriesQueryRunnerTest
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testTimeseriesWithBetweenFilter1()
|
||||
public void testTimeseriesWithBoundFilter1()
|
||||
{
|
||||
TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
|
||||
.dataSource(QueryRunnerTestHelper.dataSource)
|
||||
|
|
|
@ -0,0 +1,213 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.segment;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.query.aggregation.Aggregator;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.CountAggregatorFactory;
|
||||
import io.druid.segment.incremental.IncrementalIndex;
|
||||
import io.druid.segment.incremental.IncrementalIndexSchema;
|
||||
import io.druid.segment.incremental.IndexSizeExceededException;
|
||||
import io.druid.segment.incremental.OnheapIncrementalIndex;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
* Helps tests make segments.
|
||||
*/
|
||||
public class IndexBuilder
|
||||
{
|
||||
private static final int ROWS_PER_INDEX_FOR_MERGING = 2;
|
||||
private static final int MAX_ROWS = 50_000;
|
||||
|
||||
private IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder().withMetrics(new AggregatorFactory[]{
|
||||
new CountAggregatorFactory("count")
|
||||
}).build();
|
||||
private IndexMerger indexMerger = TestHelper.getTestIndexMerger();
|
||||
private File tmpDir;
|
||||
private IndexSpec indexSpec = new IndexSpec();
|
||||
|
||||
private final List<InputRow> rows = Lists.newArrayList();
|
||||
|
||||
private IndexBuilder()
|
||||
{
|
||||
|
||||
}
|
||||
|
||||
public static IndexBuilder create()
|
||||
{
|
||||
return new IndexBuilder();
|
||||
}
|
||||
|
||||
public IndexBuilder schema(IncrementalIndexSchema schema)
|
||||
{
|
||||
this.schema = schema;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexBuilder indexMerger(IndexMerger indexMerger)
|
||||
{
|
||||
this.indexMerger = indexMerger;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexBuilder indexSpec(IndexSpec indexSpec)
|
||||
{
|
||||
this.indexSpec = indexSpec;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexBuilder tmpDir(File tmpDir)
|
||||
{
|
||||
this.tmpDir = tmpDir;
|
||||
return this;
|
||||
}
|
||||
|
||||
public IndexBuilder add(InputRow... rows)
|
||||
{
|
||||
return add(Arrays.asList(rows));
|
||||
}
|
||||
|
||||
public IndexBuilder add(List<InputRow> rows)
|
||||
{
|
||||
this.rows.addAll(rows);
|
||||
return this;
|
||||
}
|
||||
|
||||
public IncrementalIndex buildIncrementalIndex()
|
||||
{
|
||||
return buildIncrementalIndexWithRows(schema, rows);
|
||||
}
|
||||
|
||||
public QueryableIndex buildMMappedIndex()
|
||||
{
|
||||
Preconditions.checkNotNull(indexMerger, "indexMerger");
|
||||
Preconditions.checkNotNull(tmpDir, "tmpDir");
|
||||
final IncrementalIndex incrementalIndex = buildIncrementalIndex();
|
||||
try {
|
||||
return TestHelper.getTestIndexIO().loadIndex(
|
||||
indexMerger.persist(
|
||||
incrementalIndex,
|
||||
new File(tmpDir, String.format("testIndex-%s", new Random().nextInt(Integer.MAX_VALUE))),
|
||||
indexSpec
|
||||
)
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
public QueryableIndex buildMMappedMergedIndex()
|
||||
{
|
||||
Preconditions.checkNotNull(indexMerger, "indexMerger");
|
||||
Preconditions.checkNotNull(tmpDir, "tmpDir");
|
||||
|
||||
final List<QueryableIndex> persisted = Lists.newArrayList();
|
||||
try {
|
||||
for (int i = 0; i < rows.size(); i += ROWS_PER_INDEX_FOR_MERGING) {
|
||||
persisted.add(
|
||||
TestHelper.getTestIndexIO().loadIndex(
|
||||
indexMerger.persist(
|
||||
buildIncrementalIndexWithRows(
|
||||
schema,
|
||||
rows.subList(i, Math.min(rows.size(), i + ROWS_PER_INDEX_FOR_MERGING))
|
||||
),
|
||||
new File(tmpDir, String.format("testIndex-%s", UUID.randomUUID().toString())),
|
||||
indexSpec
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
final QueryableIndex merged = TestHelper.getTestIndexIO().loadIndex(
|
||||
indexMerger.merge(
|
||||
Lists.transform(
|
||||
persisted,
|
||||
new Function<QueryableIndex, IndexableAdapter>()
|
||||
{
|
||||
@Override
|
||||
public IndexableAdapter apply(QueryableIndex input)
|
||||
{
|
||||
return new QueryableIndexIndexableAdapter(input);
|
||||
}
|
||||
}
|
||||
),
|
||||
Iterables.toArray(
|
||||
Iterables.transform(
|
||||
Arrays.asList(schema.getMetrics()),
|
||||
new Function<AggregatorFactory, AggregatorFactory>()
|
||||
{
|
||||
@Override
|
||||
public AggregatorFactory apply(AggregatorFactory input)
|
||||
{
|
||||
return input.getCombiningFactory();
|
||||
}
|
||||
}
|
||||
),
|
||||
AggregatorFactory.class
|
||||
),
|
||||
new File(tmpDir, String.format("testIndex-%s", UUID.randomUUID())),
|
||||
indexSpec
|
||||
)
|
||||
);
|
||||
for (QueryableIndex index : persisted) {
|
||||
index.close();
|
||||
}
|
||||
return merged;
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static IncrementalIndex buildIncrementalIndexWithRows(
|
||||
IncrementalIndexSchema schema,
|
||||
Iterable<InputRow> rows
|
||||
)
|
||||
{
|
||||
Preconditions.checkNotNull(schema, "schema");
|
||||
final IncrementalIndex incrementalIndex = new OnheapIncrementalIndex(
|
||||
schema,
|
||||
true,
|
||||
MAX_ROWS
|
||||
);
|
||||
for (InputRow row : rows) {
|
||||
try {
|
||||
incrementalIndex.add(row);
|
||||
}
|
||||
catch (IndexSizeExceededException e) {
|
||||
throw Throwables.propagate(e);
|
||||
}
|
||||
}
|
||||
return incrementalIndex;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,238 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.guava.Sequence;
|
||||
import com.metamx.common.guava.Sequences;
|
||||
import io.druid.common.utils.JodaUtils;
|
||||
import io.druid.granularity.QueryGranularity;
|
||||
import io.druid.query.dimension.DefaultDimensionSpec;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.filter.ValueMatcher;
|
||||
import io.druid.query.filter.ValueMatcherFactory;
|
||||
import io.druid.segment.Cursor;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.IndexMerger;
|
||||
import io.druid.segment.IndexSpec;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.QueryableIndexStorageAdapter;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.TestHelper;
|
||||
import io.druid.segment.data.BitmapSerdeFactory;
|
||||
import io.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
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.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
|
||||
public abstract class BaseFilterTest
|
||||
{
|
||||
@Rule
|
||||
public TemporaryFolder temporaryFolder = new TemporaryFolder();
|
||||
|
||||
protected StorageAdapter adapter;
|
||||
protected Closeable closeable;
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception
|
||||
{
|
||||
closeable.close();
|
||||
}
|
||||
|
||||
public static Collection<Object[]> makeConstructors()
|
||||
{
|
||||
final List<Object[]> constructors = Lists.newArrayList();
|
||||
|
||||
final Map<String, BitmapSerdeFactory> bitmapSerdeFactories = ImmutableMap.<String, BitmapSerdeFactory>of(
|
||||
"concise", new ConciseBitmapSerdeFactory(),
|
||||
"roaring", new RoaringBitmapSerdeFactory()
|
||||
);
|
||||
|
||||
final Map<String, IndexMerger> indexMergers = ImmutableMap.<String, IndexMerger>of(
|
||||
// TODO: deal with inconsistent null handling in IndexMerger
|
||||
// "IndexMerger", TestHelper.getTestIndexMerger(),
|
||||
"IndexMergerV9", TestHelper.getTestIndexMergerV9()
|
||||
);
|
||||
|
||||
final Map<String, Function<IndexBuilder, Pair<StorageAdapter, Closeable>>> finishers = ImmutableMap.of(
|
||||
"incremental", new Function<IndexBuilder, Pair<StorageAdapter, Closeable>>()
|
||||
{
|
||||
@Override
|
||||
public Pair<StorageAdapter, Closeable> apply(IndexBuilder input)
|
||||
{
|
||||
final IncrementalIndex index = input.buildIncrementalIndex();
|
||||
return Pair.<StorageAdapter, Closeable>of(
|
||||
new IncrementalIndexStorageAdapter(index),
|
||||
new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
index.close();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
},
|
||||
"mmapped", new Function<IndexBuilder, Pair<StorageAdapter, Closeable>>()
|
||||
{
|
||||
@Override
|
||||
public Pair<StorageAdapter, Closeable> apply(IndexBuilder input)
|
||||
{
|
||||
final QueryableIndex index = input.buildMMappedIndex();
|
||||
return Pair.<StorageAdapter, Closeable>of(
|
||||
new QueryableIndexStorageAdapter(index),
|
||||
new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
index.close();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
},
|
||||
"mmappedMerged", new Function<IndexBuilder, Pair<StorageAdapter, Closeable>>()
|
||||
{
|
||||
@Override
|
||||
public Pair<StorageAdapter, Closeable> apply(IndexBuilder input)
|
||||
{
|
||||
final QueryableIndex index = input.buildMMappedMergedIndex();
|
||||
return Pair.<StorageAdapter, Closeable>of(
|
||||
new QueryableIndexStorageAdapter(index),
|
||||
new Closeable()
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
index.close();
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
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()) {
|
||||
final String testName = String.format(
|
||||
"bitmaps[%s], indexMerger[%s], finisher[%s]",
|
||||
bitmapSerdeFactoryEntry.getKey(),
|
||||
indexMergerEntry.getKey(),
|
||||
finisherEntry.getKey()
|
||||
);
|
||||
final IndexBuilder indexBuilder = IndexBuilder.create()
|
||||
.indexSpec(new IndexSpec(
|
||||
bitmapSerdeFactoryEntry.getValue(),
|
||||
null,
|
||||
null
|
||||
))
|
||||
.indexMerger(indexMergerEntry.getValue());
|
||||
|
||||
constructors.add(new Object[]{testName, indexBuilder, finisherEntry.getValue()});
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return constructors;
|
||||
}
|
||||
|
||||
/**
|
||||
* Selects elements from "selectColumn" from rows matching a filter. selectColumn must be a single valued dimension.
|
||||
*/
|
||||
protected List<String> selectUsingColumn(final Filter filter, final String selectColumn)
|
||||
{
|
||||
final Sequence<Cursor> cursors = adapter.makeCursors(
|
||||
filter,
|
||||
new Interval(JodaUtils.MIN_INSTANT, JodaUtils.MAX_INSTANT),
|
||||
QueryGranularity.ALL,
|
||||
false
|
||||
);
|
||||
final Cursor cursor = Iterables.getOnlyElement(Sequences.toList(cursors, Lists.<Cursor>newArrayList()));
|
||||
final List<String> values = Lists.newArrayList();
|
||||
final DimensionSelector selector = cursor.makeDimensionSelector(
|
||||
new DefaultDimensionSpec(selectColumn, selectColumn)
|
||||
);
|
||||
|
||||
for (; !cursor.isDone(); cursor.advance()) {
|
||||
final IndexedInts row = selector.getRow();
|
||||
Preconditions.checkState(row.size() == 1);
|
||||
values.add(selector.lookupName(row.get(0)));
|
||||
}
|
||||
|
||||
return values;
|
||||
}
|
||||
|
||||
protected boolean applyFilterToValue(final Filter filter, final Comparable theValue)
|
||||
{
|
||||
return filter.makeMatcher(
|
||||
new ValueMatcherFactory()
|
||||
{
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(final String dimension, final Comparable value)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return Objects.equals(value, theValue);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public ValueMatcher makeValueMatcher(String dimension, final Predicate predicate)
|
||||
{
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return predicate.apply(theValue);
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
||||
).matches();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,320 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.Pair;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.MapInputRowParser;
|
||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.query.filter.BoundDimFilter;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class BoundFilterTest extends BaseFilterTest
|
||||
{
|
||||
private static final String TIMESTAMP_COLUMN = "timestamp";
|
||||
|
||||
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||
new TimeAndDimsParseSpec(
|
||||
new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")),
|
||||
new DimensionsSpec(null, null, null)
|
||||
)
|
||||
);
|
||||
|
||||
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "a", "dim1", "", "dim2", ImmutableList.of("a", "b"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "b", "dim1", "10", "dim2", ImmutableList.of())),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "c", "dim1", "2", "dim2", ImmutableList.of(""))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "d", "dim1", "1", "dim2", ImmutableList.of("a"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "e", "dim1", "def", "dim2", ImmutableList.of("c"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "f", "dim1", "abc"))
|
||||
);
|
||||
|
||||
private final IndexBuilder indexBuilder;
|
||||
private final Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher;
|
||||
|
||||
public BoundFilterTest(
|
||||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher
|
||||
)
|
||||
{
|
||||
this.indexBuilder = indexBuilder;
|
||||
this.finisher = finisher;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException
|
||||
{
|
||||
final Pair<StorageAdapter, Closeable> pair = finisher.apply(
|
||||
indexBuilder.tmpDir(temporaryFolder.newFolder()).add(ROWS)
|
||||
);
|
||||
this.adapter = pair.lhs;
|
||||
this.closeable = pair.rhs;
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> constructorFeeder() throws IOException
|
||||
{
|
||||
return makeConstructors();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchEverything()
|
||||
{
|
||||
final List<BoundFilter> filters = ImmutableList.of(
|
||||
new BoundFilter(new BoundDimFilter("dim0", "", "z", false, false, false)),
|
||||
new BoundFilter(new BoundDimFilter("dim1", "", "z", false, false, false)),
|
||||
new BoundFilter(new BoundDimFilter("dim2", "", "z", false, false, false)),
|
||||
new BoundFilter(new BoundDimFilter("dim3", "", "z", false, false, false))
|
||||
);
|
||||
|
||||
for (BoundFilter filter : filters) {
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(filter));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchNull()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim0", "", "", false, false, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "", "", false, false, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(1, 2, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim2", "", "", false, false, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchMissingColumn()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0, 1, 2, 3, 4, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim3", "", "", false, false, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim3", "", "", true, false, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim3", "", "", false, true, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0, 1, 2, 3, 4, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim3", "", null, false, true, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0, 1, 2, 3, 4, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim3", null, "", false, false, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim3", null, "", false, true, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchTooStrict()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "abc", "abc", true, false, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "abc", "abc", true, true, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "abc", "abc", false, true, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchExactlySingleValue()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "abc", "abc", false, false, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchSurroundingSingleValue()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "ab", "abd", true, true, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchNoUpperLimit()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(4, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "ab", null, true, true, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchNoLowerLimit()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0, 1, 2, 3, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", null, "abd", true, true, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLexicographicMatchNumbers()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(1, 2, 3),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "1", "3", false, false, false)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(1, 2),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "1", "3", true, true, false)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlphaNumericMatchNull()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim0", "", "", false, false, true)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "", "", false, false, true)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(1, 2, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim2", "", "", false, false, true)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0, 1, 2, 3, 4, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim3", "", "", false, false, true)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlphaNumericMatchTooStrict()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "2", "2", true, false, true)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "2", "2", true, true, true)))
|
||||
);
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "2", "2", false, true, true)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlphaNumericMatchExactlySingleValue()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(2),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "2", "2", false, false, true)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlphaNumericMatchSurroundingSingleValue()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(2),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "1", "3", true, true, true)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlphaNumericMatchNoUpperLimit()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(1, 2, 4, 5),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", "1", null, true, true, true)))
|
||||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAlphaNumericMatchNoLowerLimit()
|
||||
{
|
||||
Assert.assertEquals(
|
||||
ImmutableList.of(0, 3),
|
||||
select(new BoundFilter(new BoundDimFilter("dim1", null, "2", true, true, true)))
|
||||
);
|
||||
}
|
||||
|
||||
private List<Integer> select(final Filter filter)
|
||||
{
|
||||
return Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
selectUsingColumn(filter, "dim0"),
|
||||
new Function<String, Integer>()
|
||||
{
|
||||
@Override
|
||||
public Integer apply(String input)
|
||||
{
|
||||
Preconditions.checkArgument(input.length() == 1);
|
||||
return ((int) input.charAt(0)) - ((int) 'a');
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -32,13 +32,20 @@ import io.druid.query.extraction.ExtractionFn;
|
|||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.DimFilters;
|
||||
import io.druid.query.filter.ExtractionDimFilter;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.data.ArrayIndexed;
|
||||
import io.druid.segment.data.BitmapSerdeFactory;
|
||||
import io.druid.segment.data.ConciseBitmapSerdeFactory;
|
||||
import io.druid.segment.data.GenericIndexed;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.data.RoaringBitmapSerdeFactory;
|
||||
import io.druid.segment.serde.BitmapIndexColumnPartSupplier;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
@ -61,20 +68,22 @@ public class ExtractionDimFilterTest
|
|||
public static Iterable<Object[]> constructorFeeder()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new Object[]{new ConciseBitmapFactory()},
|
||||
new Object[]{new RoaringBitmapFactory()}
|
||||
new Object[]{new ConciseBitmapFactory(), new ConciseBitmapSerdeFactory()},
|
||||
new Object[]{new RoaringBitmapFactory(), new RoaringBitmapSerdeFactory()}
|
||||
);
|
||||
}
|
||||
|
||||
public ExtractionDimFilterTest(BitmapFactory bitmapFactory)
|
||||
public ExtractionDimFilterTest(BitmapFactory bitmapFactory, BitmapSerdeFactory bitmapSerdeFactory)
|
||||
{
|
||||
final MutableBitmap mutableBitmap = bitmapFactory.makeEmptyMutableBitmap();
|
||||
mutableBitmap.add(1);
|
||||
this.foo1BitMap = bitmapFactory.makeImmutableBitmap(mutableBitmap);
|
||||
this.factory = bitmapFactory;
|
||||
this.serdeFactory = bitmapSerdeFactory;
|
||||
}
|
||||
|
||||
private final BitmapFactory factory;
|
||||
private final BitmapSerdeFactory serdeFactory;
|
||||
private final ImmutableBitmap foo1BitMap;
|
||||
|
||||
private final BitmapIndexSelector BITMAP_INDEX_SELECTOR = new BitmapIndexSelector()
|
||||
|
@ -104,6 +113,16 @@ public class ExtractionDimFilterTest
|
|||
return "foo1".equals(value) ? foo1BitMap : null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public BitmapIndex getBitmapIndex(String dimension)
|
||||
{
|
||||
return new BitmapIndexColumnPartSupplier(
|
||||
factory,
|
||||
GenericIndexed.fromIterable(Arrays.asList(foo1BitMap), serdeFactory.getObjectStrategy()),
|
||||
GenericIndexed.fromIterable(Arrays.asList("foo1"), GenericIndexed.STRING_STRATEGY)
|
||||
).get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImmutableRTree getSpatialIndex(String dimension)
|
||||
{
|
||||
|
|
|
@ -0,0 +1,161 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.Pair;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.MapInputRowParser;
|
||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class NotFilterTest extends BaseFilterTest
|
||||
{
|
||||
private static final String TIMESTAMP_COLUMN = "timestamp";
|
||||
|
||||
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||
new TimeAndDimsParseSpec(
|
||||
new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")),
|
||||
new DimensionsSpec(null, null, null)
|
||||
)
|
||||
);
|
||||
|
||||
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "a", "dim1", "", "dim2", ImmutableList.of("a", "b"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "b", "dim1", "10", "dim2", ImmutableList.of())),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "c", "dim1", "2", "dim2", ImmutableList.of(""))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "d", "dim1", "1", "dim2", ImmutableList.of("a"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "e", "dim1", "def", "dim2", ImmutableList.of("c"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "f", "dim1", "abc"))
|
||||
);
|
||||
|
||||
private final IndexBuilder indexBuilder;
|
||||
private final Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher;
|
||||
|
||||
public NotFilterTest(
|
||||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher
|
||||
)
|
||||
{
|
||||
this.indexBuilder = indexBuilder;
|
||||
this.finisher = finisher;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException
|
||||
{
|
||||
final Pair<StorageAdapter, Closeable> pair = finisher.apply(
|
||||
indexBuilder.tmpDir(temporaryFolder.newFolder()).add(ROWS)
|
||||
);
|
||||
this.adapter = pair.lhs;
|
||||
this.closeable = pair.rhs;
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> constructorFeeder() throws IOException
|
||||
{
|
||||
return makeConstructors();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleValueStringColumnWithoutNulls()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim0", null))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim0", ""))));
|
||||
Assert.assertEquals(ImmutableList.of(1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim0", "a"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim0", "b"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleValueStringColumnWithNulls()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim1", null))));
|
||||
Assert.assertEquals(ImmutableList.of(1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim1", ""))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim1", "10"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim1", "2"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 4, 5), select(new NotFilter(new SelectorFilter("dim1", "1"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 5), select(new NotFilter(new SelectorFilter("dim1", "def"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4), select(new NotFilter(new SelectorFilter("dim1", "abc"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim1", "ab"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringColumn()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(0, 3, 4), select(new NotFilter(new SelectorFilter("dim2", null))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 3, 4), select(new NotFilter(new SelectorFilter("dim2", ""))));
|
||||
Assert.assertEquals(ImmutableList.of(1, 2, 4, 5), select(new NotFilter(new SelectorFilter("dim2", "a"))));
|
||||
Assert.assertEquals(ImmutableList.of(1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim2", "b"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 5), select(new NotFilter(new SelectorFilter("dim2", "c"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim2", "d"))));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingColumn()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(), select(new NotFilter(new SelectorFilter("dim3", null))));
|
||||
Assert.assertEquals(ImmutableList.of(), select(new NotFilter(new SelectorFilter("dim3", ""))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim3", "a"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim3", "b"))));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new NotFilter(new SelectorFilter("dim3", "c"))));
|
||||
}
|
||||
|
||||
private List<Integer> select(final Filter filter)
|
||||
{
|
||||
return Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
selectUsingColumn(filter, "dim0"),
|
||||
new Function<String, Integer>()
|
||||
{
|
||||
@Override
|
||||
public Integer apply(String input)
|
||||
{
|
||||
Preconditions.checkArgument(input.length() == 1);
|
||||
return ((int) input.charAt(0)) - ((int) 'a');
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,161 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.Pair;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.DimensionsSpec;
|
||||
import io.druid.data.input.impl.InputRowParser;
|
||||
import io.druid.data.input.impl.MapInputRowParser;
|
||||
import io.druid.data.input.impl.TimeAndDimsParseSpec;
|
||||
import io.druid.data.input.impl.TimestampSpec;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.segment.IndexBuilder;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import org.joda.time.DateTime;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
import org.junit.runners.Parameterized;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class SelectorFilterTest extends BaseFilterTest
|
||||
{
|
||||
private static final String TIMESTAMP_COLUMN = "timestamp";
|
||||
|
||||
private static final InputRowParser<Map<String, Object>> PARSER = new MapInputRowParser(
|
||||
new TimeAndDimsParseSpec(
|
||||
new TimestampSpec(TIMESTAMP_COLUMN, "iso", new DateTime("2000")),
|
||||
new DimensionsSpec(null, null, null)
|
||||
)
|
||||
);
|
||||
|
||||
private static final List<InputRow> ROWS = ImmutableList.of(
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "a", "dim1", "", "dim2", ImmutableList.of("a", "b"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "b", "dim1", "10", "dim2", ImmutableList.of())),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "c", "dim1", "2", "dim2", ImmutableList.of(""))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "d", "dim1", "1", "dim2", ImmutableList.of("a"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "e", "dim1", "def", "dim2", ImmutableList.of("c"))),
|
||||
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "f", "dim1", "abc"))
|
||||
);
|
||||
|
||||
private final IndexBuilder indexBuilder;
|
||||
private final Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher;
|
||||
|
||||
public SelectorFilterTest(
|
||||
String testName,
|
||||
IndexBuilder indexBuilder,
|
||||
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher
|
||||
)
|
||||
{
|
||||
this.indexBuilder = indexBuilder;
|
||||
this.finisher = finisher;
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException
|
||||
{
|
||||
final Pair<StorageAdapter, Closeable> pair = finisher.apply(
|
||||
indexBuilder.tmpDir(temporaryFolder.newFolder()).add(ROWS)
|
||||
);
|
||||
this.adapter = pair.lhs;
|
||||
this.closeable = pair.rhs;
|
||||
}
|
||||
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Collection<Object[]> constructorFeeder() throws IOException
|
||||
{
|
||||
return makeConstructors();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleValueStringColumnWithoutNulls()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(), select(new SelectorFilter("dim0", null)));
|
||||
Assert.assertEquals(ImmutableList.of(), select(new SelectorFilter("dim0", "")));
|
||||
Assert.assertEquals(ImmutableList.of(0), select(new SelectorFilter("dim0", "a")));
|
||||
Assert.assertEquals(ImmutableList.of(1), select(new SelectorFilter("dim0", "b")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleValueStringColumnWithNulls()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(0), select(new SelectorFilter("dim1", null)));
|
||||
Assert.assertEquals(ImmutableList.of(0), select(new SelectorFilter("dim1", "")));
|
||||
Assert.assertEquals(ImmutableList.of(1), select(new SelectorFilter("dim1", "10")));
|
||||
Assert.assertEquals(ImmutableList.of(2), select(new SelectorFilter("dim1", "2")));
|
||||
Assert.assertEquals(ImmutableList.of(3), select(new SelectorFilter("dim1", "1")));
|
||||
Assert.assertEquals(ImmutableList.of(4), select(new SelectorFilter("dim1", "def")));
|
||||
Assert.assertEquals(ImmutableList.of(5), select(new SelectorFilter("dim1", "abc")));
|
||||
Assert.assertEquals(ImmutableList.of(), select(new SelectorFilter("dim1", "ab")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMultiValueStringColumn()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(1, 2, 5), select(new SelectorFilter("dim2", null)));
|
||||
Assert.assertEquals(ImmutableList.of(1, 2, 5), select(new SelectorFilter("dim2", "")));
|
||||
Assert.assertEquals(ImmutableList.of(0, 3), select(new SelectorFilter("dim2", "a")));
|
||||
Assert.assertEquals(ImmutableList.of(0), select(new SelectorFilter("dim2", "b")));
|
||||
Assert.assertEquals(ImmutableList.of(4), select(new SelectorFilter("dim2", "c")));
|
||||
Assert.assertEquals(ImmutableList.of(), select(new SelectorFilter("dim2", "d")));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testMissingColumn()
|
||||
{
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new SelectorFilter("dim3", null)));
|
||||
Assert.assertEquals(ImmutableList.of(0, 1, 2, 3, 4, 5), select(new SelectorFilter("dim3", "")));
|
||||
Assert.assertEquals(ImmutableList.of(), select(new SelectorFilter("dim3", "a")));
|
||||
Assert.assertEquals(ImmutableList.of(), select(new SelectorFilter("dim3", "b")));
|
||||
Assert.assertEquals(ImmutableList.of(), select(new SelectorFilter("dim3", "c")));
|
||||
}
|
||||
|
||||
private List<Integer> select(final Filter filter)
|
||||
{
|
||||
return Lists.newArrayList(
|
||||
Iterables.transform(
|
||||
selectUsingColumn(filter, "dim0"),
|
||||
new Function<String, Integer>()
|
||||
{
|
||||
@Override
|
||||
public Integer apply(String input)
|
||||
{
|
||||
Preconditions.checkArgument(input.length() == 1);
|
||||
return ((int) input.charAt(0)) - ((int) 'a');
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue