Two-stage filtering (#3018)

* Two-stage filtering

* PR comment
This commit is contained in:
Jonathan Wei 2016-06-22 16:08:21 -07:00 committed by Nishant
parent ebd654228b
commit 24860a1391
31 changed files with 2279 additions and 83 deletions

6
NOTICE
View File

@ -10,3 +10,9 @@ This product contains a modified version of Andrew Duffy's java-alphanum library
* https://github.com/amjjd/java-alphanum/blob/5c036e2e492cc7f3b7bcdebd46b8f9e2a87927e5/LICENSE.txt (Apache License, Version 2.0)
* HOMEPAGE:
* https://github.com/amjjd/java-alphanum
This product contains conjunctive normal form conversion code adapted from Apache Hive
* LICENSE:
* https://github.com/apache/hive/blob/branch-2.0/LICENSE (Apache License, Version 2.0)
* HOMEPAGE:
* https://github.com/apache/hive

View File

@ -0,0 +1,531 @@
/*
* 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.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Strings;
import com.google.common.collect.Lists;
import com.google.common.hash.Hashing;
import com.google.common.io.Files;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import com.metamx.common.logger.Logger;
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
import io.druid.benchmark.datagen.BenchmarkSchemas;
import io.druid.data.input.InputRow;
import io.druid.data.input.impl.DimensionsSpec;
import io.druid.granularity.QueryGranularities;
import io.druid.jackson.DefaultObjectMapper;
import io.druid.js.JavaScriptConfig;
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.DimFilter;
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.IndexIO;
import io.druid.segment.IndexMergerV9;
import io.druid.segment.IndexSpec;
import io.druid.segment.LongColumnSelector;
import io.druid.segment.QueryableIndex;
import io.druid.segment.QueryableIndexStorageAdapter;
import io.druid.segment.StorageAdapter;
import io.druid.segment.column.ColumnConfig;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.filter.AndFilter;
import io.druid.segment.filter.DimensionPredicateFilter;
import io.druid.segment.filter.Filters;
import io.druid.segment.filter.OrFilter;
import io.druid.segment.filter.SelectorFilter;
import io.druid.segment.incremental.IncrementalIndex;
import io.druid.segment.incremental.IncrementalIndexSchema;
import io.druid.segment.incremental.OnheapIncrementalIndex;
import io.druid.segment.serde.ComplexMetrics;
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 org.openjdk.jmh.infra.Blackhole;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.concurrent.TimeUnit;
@State(Scope.Benchmark)
@Fork(jvmArgsPrepend = "-server", value = 1)
@Warmup(iterations = 10)
@Measurement(iterations = 25)
public class FilterPartitionBenchmark
{
@Param({"750000"})
private int rowsPerSegment;
@Param({"basic"})
private String schema;
private static final Logger log = new Logger(FilterPartitionBenchmark.class);
private static final int RNG_SEED = 9999;
private static final IndexMergerV9 INDEX_MERGER_V9;
private static final IndexIO INDEX_IO;
public static final ObjectMapper JSON_MAPPER;
private IncrementalIndex incIndex;
private QueryableIndex qIndex;
private File indexFile;
private BenchmarkSchemaInfo schemaInfo;
private static String JS_FN = "function(str) { return 'super-' + str; }";
private static ExtractionFn JS_EXTRACTION_FN = new JavaScriptExtractionFn(JS_FN, false, JavaScriptConfig.getDefault());
static {
JSON_MAPPER = new DefaultObjectMapper();
INDEX_IO = new IndexIO(
JSON_MAPPER,
new ColumnConfig()
{
@Override
public int columnCacheSizeBytes()
{
return 0;
}
}
);
INDEX_MERGER_V9 = new IndexMergerV9(JSON_MAPPER, INDEX_IO);
}
@Setup
public void setup() throws IOException
{
log.info("SETUP CALLED AT " + System.currentTimeMillis());
if (ComplexMetrics.getSerdeForType("hyperUnique") == null) {
ComplexMetrics.registerSerde("hyperUnique", new HyperUniquesSerde(Hashing.murmur3_128()));
}
schemaInfo = BenchmarkSchemas.SCHEMA_MAP.get(schema);
BenchmarkDataGenerator gen = new BenchmarkDataGenerator(
schemaInfo.getColumnSchemas(),
RNG_SEED,
schemaInfo.getDataInterval(),
rowsPerSegment
);
incIndex = makeIncIndex();
for (int j = 0; j < rowsPerSegment; j++) {
InputRow row = gen.nextRow();
if (j % 10000 == 0) {
log.info(j + " rows generated.");
}
incIndex.add(row);
}
File tmpFile = Files.createTempDir();
log.info("Using temp dir: " + tmpFile.getAbsolutePath());
tmpFile.deleteOnExit();
indexFile = INDEX_MERGER_V9.persist(
incIndex,
tmpFile,
new IndexSpec()
);
qIndex = INDEX_IO.loadIndex(indexFile);
}
private IncrementalIndex makeIncIndex()
{
return new OnheapIncrementalIndex(
new IncrementalIndexSchema.Builder()
.withQueryGranularity(QueryGranularities.NONE)
.withMetrics(schemaInfo.getAggsArray())
.withDimensionsSpec(new DimensionsSpec(null, null, null))
.build(),
true,
false,
true,
rowsPerSegment
);
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void stringRead(Blackhole blackhole) throws Exception
{
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(null, schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void longRead(Blackhole blackhole) throws Exception
{
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(null, schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<Long>> longListSeq = readCursorsLong(cursors, blackhole);
List<Long> strings = Sequences.toList(Sequences.limit(longListSeq, 1), Lists.<List<Long>>newArrayList()).get(0);
for (Long st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readWithPreFilter(Blackhole blackhole) throws Exception
{
Filter filter = new SelectorFilter("dimSequential", "199");
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(filter, schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readWithPostFilter(Blackhole blackhole) throws Exception
{
Filter filter = new NoBitmapSelectorFilter("dimSequential", "199");
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(filter, schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readWithExFnPreFilter(Blackhole blackhole) throws Exception
{
Filter filter = new SelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter();
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(filter, schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readWithExFnPostFilter(Blackhole blackhole) throws Exception
{
Filter filter = new NoBitmapSelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter();
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(filter, schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readOrFilter(Blackhole blackhole) throws Exception
{
Filter filter = new NoBitmapSelectorFilter("dimSequential", "199");
Filter filter2 = new AndFilter(Arrays.<Filter>asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar")));
Filter orFilter = new OrFilter(Arrays.<Filter>asList(filter, filter2));
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(orFilter, schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readOrFilterCNF(Blackhole blackhole) throws Exception
{
Filter filter = new NoBitmapSelectorFilter("dimSequential", "199");
Filter filter2 = new AndFilter(Arrays.<Filter>asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar")));
Filter orFilter = new OrFilter(Arrays.<Filter>asList(filter, filter2));
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(Filters.convertToCNF(orFilter), schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readComplexOrFilter(Blackhole blackhole) throws Exception
{
DimFilter dimFilter1 = new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dimSequential", "199", null),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dimMultivalEnumerated2", "Corundum", null),
new SelectorDimFilter("dimMultivalEnumerated", "Bar", null)
)
))
);
DimFilter dimFilter2 = new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dimSequential", "299", null),
new SelectorDimFilter("dimSequential", "399", null),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dimMultivalEnumerated2", "Xylophone", null),
new SelectorDimFilter("dimMultivalEnumerated", "Foo", null)
)
))
);
DimFilter dimFilter3 = new OrDimFilter(Arrays.<DimFilter>asList(
dimFilter1,
dimFilter2,
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dimMultivalEnumerated2", "Orange", null),
new SelectorDimFilter("dimMultivalEnumerated", "World", null)
)
))
);
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(dimFilter3.toFilter(), schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
@Benchmark
@BenchmarkMode(Mode.AverageTime)
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void readComplexOrFilterCNF(Blackhole blackhole) throws Exception
{
DimFilter dimFilter1 = new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dimSequential", "199", null),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dimMultivalEnumerated2", "Corundum", null),
new SelectorDimFilter("dimMultivalEnumerated", "Bar", null)
)
))
);
DimFilter dimFilter2 = new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dimSequential", "299", null),
new SelectorDimFilter("dimSequential", "399", null),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dimMultivalEnumerated2", "Xylophone", null),
new SelectorDimFilter("dimMultivalEnumerated", "Foo", null)
)
))
);
DimFilter dimFilter3 = new OrDimFilter(Arrays.<DimFilter>asList(
dimFilter1,
dimFilter2,
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dimMultivalEnumerated2", "Orange", null),
new SelectorDimFilter("dimMultivalEnumerated", "World", null)
)
))
);
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
Sequence<Cursor> cursors = sa.makeCursors(Filters.convertToCNF(dimFilter3.toFilter()), schemaInfo.getDataInterval(), QueryGranularities.ALL, false);
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
List<String> strings = Sequences.toList(Sequences.limit(stringListSeq, 1), Lists.<List<String>>newArrayList()).get(0);
for (String st : strings) {
blackhole.consume(st);
}
}
private Sequence<List<String>> readCursors(Sequence<Cursor> cursors, final Blackhole blackhole)
{
return Sequences.map(
cursors,
new Function<Cursor, List<String>>()
{
@Override
public List<String> apply(Cursor input)
{
List<String> strings = new ArrayList<String>();
List<DimensionSelector> selectors = new ArrayList<>();
selectors.add(input.makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null)));
//selectors.add(input.makeDimensionSelector(new DefaultDimensionSpec("dimB", null)));
while (!input.isDone()) {
for (DimensionSelector selector : selectors) {
IndexedInts row = selector.getRow();
blackhole.consume(selector.lookupName(row.get(0)));
//strings.add(selector.lookupName(row.get(0)));
}
input.advance();
}
return strings;
}
}
);
}
private Sequence<List<Long>> readCursorsLong(Sequence<Cursor> cursors, final Blackhole blackhole)
{
return Sequences.map(
cursors,
new Function<Cursor, List<Long>>()
{
@Override
public List<Long> apply(Cursor input)
{
List<Long> longvals = new ArrayList<Long>();
LongColumnSelector selector = input.makeLongColumnSelector("sumLongSequential");
while (!input.isDone()) {
long rowval = selector.get();
blackhole.consume(rowval);
input.advance();
}
return longvals;
}
}
);
}
private class NoBitmapSelectorFilter extends SelectorFilter
{
public NoBitmapSelectorFilter(
String dimension,
String value
)
{
super(dimension, value);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return false;
}
}
private class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter
{
public NoBitmapDimensionPredicateFilter(
final String dimension,
final Predicate<String> predicate,
final ExtractionFn extractionFn
)
{
super(dimension, predicate, extractionFn);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return false;
}
}
private class NoBitmapSelectorDimFilter extends SelectorDimFilter
{
public NoBitmapSelectorDimFilter(
String dimension,
String value,
ExtractionFn extractionFn
)
{
super(dimension, value, extractionFn);
}
@Override
public Filter toFilter()
{
ExtractionFn extractionFn = getExtractionFn();
String dimension = getDimension();
final String value = getValue();
if (extractionFn == null) {
return new NoBitmapSelectorFilter(dimension, value);
} else {
final String valueOrNull = Strings.emptyToNull(value);
final Predicate<String> predicate = new Predicate<String>()
{
@Override
public boolean apply(String input)
{
return Objects.equals(valueOrNull, input);
}
@Override
public String toString()
{
return value;
}
};
return new NoBitmapDimensionPredicateFilter(dimension, predicate, extractionFn);
}
}
}
}

View File

@ -0,0 +1,48 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.filter;
import java.util.List;
public interface BooleanFilter extends Filter
{
public List<Filter> getFilters();
/**
* Get a ValueMatcher that applies this filter to row values.
*
* Unlike makeMatcher(ValueMatcherFactory), this method allows the Filter to utilize bitmap indexes.
*
* An implementation should either:
* - return a ValueMatcher that checks row values, using the provided ValueMatcherFactory
* - or, if possible, get a bitmap index for this filter using the BitmapIndexSelector, and
* return a ValueMatcher that checks the current row offset, created using the bitmap index.
*
* @param selector Object used to retrieve bitmap indexes
* @param valueMatcherFactory Object used to create ValueMatchers
* @param rowOffsetMatcherFactory Object used to create RowOffsetMatchers
* @return ValueMatcher that applies this filter
*/
public ValueMatcher makeMatcher(
BitmapIndexSelector selector,
ValueMatcherFactory valueMatcherFactory,
RowOffsetMatcherFactory rowOffsetMatcherFactory
);
}

View File

@ -25,6 +25,30 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
*/
public interface Filter
{
/**
* Get a bitmap index, indicating rows that match this filter.
*
* @param selector Object used to retrieve bitmap indexes
* @return A bitmap indicating rows that match this filter.
*/
public ImmutableBitmap getBitmapIndex(BitmapIndexSelector selector);
/**
* Get a ValueMatcher that applies this filter to row values.
*
* @param factory Object used to create ValueMatchers
* @return ValueMatcher that applies this filter to row values.
*/
public ValueMatcher makeMatcher(ValueMatcherFactory factory);
/**
* Indicates whether this filter can return a bitmap index for filtering, based on
* the information provided by the input BitmapIndexSelector.
*
* @param selector Object used to retrieve bitmap indexes
* @return true if this Filter can provide a bitmap index using the selector, false otherwise
*/
public boolean supportsBitmapIndex(BitmapIndexSelector selector);
}

View File

@ -0,0 +1,29 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package io.druid.query.filter;
import com.metamx.collections.bitmap.ImmutableBitmap;
/**
*/
public interface RowOffsetMatcherFactory
{
public ValueMatcher makeRowOffsetMatcher(ImmutableBitmap bitmap);
}

View File

@ -92,6 +92,12 @@ public class SelectorDimFilter implements DimFilter
{
return Objects.equals(valueOrNull, input);
}
@Override
public String toString()
{
return value;
}
};
return new DimensionPredicateFilter(dimension, predicate, extractionFn);
}

View File

@ -40,10 +40,12 @@ import io.druid.collections.StupidPool;
import io.druid.data.input.MapBasedRow;
import io.druid.data.input.Row;
import io.druid.guice.annotations.Global;
import io.druid.query.Query;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.aggregation.BufferAggregator;
import io.druid.query.aggregation.PostAggregator;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.filter.Filter;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.StorageAdapter;
@ -94,8 +96,10 @@ public class GroupByQueryEngine
throw new IAE("Should only have one interval, got[%s]", intervals);
}
Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimFilter()));
final Sequence<Cursor> cursors = storageAdapter.makeCursors(
Filters.toFilter(query.getDimFilter()),
filter,
intervals.get(0),
query.getGranularity(),
false

View File

@ -85,7 +85,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
}
final SearchQuery query = (SearchQuery) input;
final Filter filter = Filters.toFilter(query.getDimensionsFilter());
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
final List<DimensionSpec> dimensions = query.getDimensions();
final SearchQuerySpec searchQuerySpec = query.getQuery();
final int limit = query.getLimit();

View File

@ -30,6 +30,7 @@ import io.druid.query.QueryRunnerHelper;
import io.druid.query.Result;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.filter.Filter;
import io.druid.segment.Cursor;
import io.druid.segment.DimensionSelector;
import io.druid.segment.LongColumnSelector;
@ -82,10 +83,12 @@ public class SelectQueryEngine
// should be rewritten with given interval
final String segmentId = DataSegmentUtils.withInterval(dataSource, segment.getIdentifier(), intervals.get(0));
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
return QueryRunnerHelper.makeCursorBasedQuery(
adapter,
query.getQuerySegmentSpec().getIntervals(),
Filters.toFilter(query.getDimensionsFilter()),
filter,
query.isDescending(),
query.getGranularity(),
new Function<Cursor, Result<SelectResultValue>>()

View File

@ -25,6 +25,7 @@ import io.druid.query.QueryRunnerHelper;
import io.druid.query.Result;
import io.druid.query.aggregation.Aggregator;
import io.druid.query.aggregation.AggregatorFactory;
import io.druid.query.filter.Filter;
import io.druid.segment.Cursor;
import io.druid.segment.SegmentMissingException;
import io.druid.segment.StorageAdapter;
@ -44,10 +45,12 @@ public class TimeseriesQueryEngine
);
}
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
return QueryRunnerHelper.makeCursorBasedQuery(
adapter,
query.getQuerySegmentSpec().getIntervals(),
Filters.toFilter(query.getDimensionsFilter()),
filter,
query.isDescending(),
query.getGranularity(),
new Function<Cursor, Result<TimeseriesResultValue>>()

View File

@ -64,7 +64,7 @@ public class TopNQueryEngine
}
final List<Interval> queryIntervals = query.getQuerySegmentSpec().getIntervals();
final Filter filter = Filters.toFilter(query.getDimensionsFilter());
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
final QueryGranularity granularity = query.getGranularity();
final Function<Cursor, Result<TopNResultValue>> mapFn = getMapFn(query, adapter);

View File

@ -40,6 +40,21 @@ public class BitmapOffset implements Offset
private volatile int val;
public static IntIterator getReverseBitmapOffsetIterator(ImmutableBitmap bitmapIndex)
{
ImmutableBitmap roaringBitmap = bitmapIndex;
if (!(bitmapIndex instanceof WrappedImmutableRoaringBitmap)) {
final BitmapFactory factory = RoaringBitmapSerdeFactory.bitmapFactory;
final MutableBitmap bitmap = factory.makeEmptyMutableBitmap();
final IntIterator iterator = bitmapIndex.iterator();
while (iterator.hasNext()) {
bitmap.add(iterator.next());
}
roaringBitmap = factory.makeImmutableBitmap(bitmap);
}
return ((WrappedImmutableRoaringBitmap) roaringBitmap).getBitmap().getReverseIntIterator();
}
public BitmapOffset(BitmapFactory bitmapFactory, ImmutableBitmap bitmapIndex, boolean descending)
{
this.bitmapFactory = bitmapFactory;
@ -53,18 +68,9 @@ public class BitmapOffset implements Offset
{
if (!descending) {
return bitmapIndex.iterator();
} else {
return getReverseBitmapOffsetIterator(bitmapIndex);
}
ImmutableBitmap roaringBitmap = bitmapIndex;
if (!(bitmapIndex instanceof WrappedImmutableRoaringBitmap)) {
final BitmapFactory factory = RoaringBitmapSerdeFactory.bitmapFactory;
final MutableBitmap bitmap = factory.makeEmptyMutableBitmap();
final IntIterator iterator = bitmapIndex.iterator();
while (iterator.hasNext()) {
bitmap.add(iterator.next());
}
roaringBitmap = factory.makeImmutableBitmap(bitmap);
}
return ((WrappedImmutableRoaringBitmap) roaringBitmap).getBitmap().getReverseIntIterator();
}
private BitmapOffset(BitmapOffset otherOffset)

View File

@ -115,7 +115,56 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
public BitmapIndex getBitmapIndex(String dimension)
{
final Column column = index.getColumn(dimension);
if (column != null && column.getCapabilities().hasBitmapIndexes()) {
if (column == null) {
// Create a BitmapIndex for null columns so that filters applied to null columns can use
// bitmap indexes. Filters check for the presence of a bitmap index, this is used to determine
// whether the filter is applied in the pre or post filtering stage.
return new BitmapIndex() {
@Override
public int getCardinality()
{
return 1;
}
@Override
public String getValue(int index)
{
return null;
}
@Override
public boolean hasNulls()
{
return true;
}
@Override
public BitmapFactory getBitmapFactory()
{
return bitmapFactory;
}
@Override
public int getIndex(String value)
{
// Return -2 for non-null values to match what the BitmapIndex implementation in BitmapIndexColumnPartSupplier
// would return for getIndex() when there is only a single index, for the null value.
// i.e., return an 'insertion point' of 1 for non-null values (see BitmapIndex interface)
return Strings.isNullOrEmpty(value) ? 0 : -2;
}
@Override
public ImmutableBitmap getBitmap(int idx)
{
if (idx == 0) {
return bitmapFactory.complement(bitmapFactory.makeEmptyImmutableBitmap(), getNumRows());
} else {
return bitmapFactory.makeEmptyImmutableBitmap();
}
}
};
} else if (column.getCapabilities().hasBitmapIndexes()) {
return column.getBitmapIndex();
} else {
return null;
@ -135,7 +184,7 @@ public class ColumnSelectorBitmapIndexSelector implements BitmapIndexSelector
}
if (!column.getCapabilities().hasBitmapIndexes()) {
return bitmapFactory.makeEmptyImmutableBitmap();
return null;
}
final BitmapIndex bitmapIndex = column.getBitmapIndex();

View File

@ -20,6 +20,7 @@
package io.druid.segment;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.base.Strings;
import com.google.common.collect.ImmutableList;
@ -27,14 +28,20 @@ import com.google.common.collect.Iterators;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.collect.Sets;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.guava.Sequence;
import com.metamx.common.guava.Sequences;
import io.druid.granularity.QueryGranularity;
import io.druid.query.QueryInterruptedException;
import io.druid.query.dimension.DefaultDimensionSpec;
import io.druid.query.dimension.DimensionSpec;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BooleanFilter;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.column.BitmapIndex;
import io.druid.segment.column.Column;
import io.druid.segment.column.ColumnCapabilities;
@ -45,12 +52,17 @@ import io.druid.segment.column.ValueType;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.IndexedInts;
import io.druid.segment.data.Offset;
import io.druid.segment.filter.AndFilter;
import io.druid.segment.filter.BooleanValueMatcher;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.roaringbitmap.IntIterator;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
/**
@ -172,7 +184,11 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
@Override
public ColumnCapabilities getColumnCapabilities(String column)
{
return index.getColumn(column).getCapabilities();
Column columnObj = index.getColumn(column);
if (columnObj == null) {
return null;
}
return columnObj.getCapabilities();
}
@Override
@ -213,16 +229,73 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
actualInterval = actualInterval.withEnd(dataInterval.getEnd());
}
final ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
index.getBitmapFactoryForDimensions(),
index
);
/**
* Filters can be applied in two stages:
* pre-filtering: Use bitmap indexes to prune the set of rows to be scanned.
* post-filtering: Iterate through rows and apply the filter to the row values
*
* The pre-filter and post-filter step have an implicit AND relationship. (i.e., final rows are those that
* were not pruned AND those that matched the filter during row scanning)
*
* An AND filter can have its subfilters partitioned across the two steps. The subfilters that can be
* processed entirely with bitmap indexes (subfilter returns true for supportsBitmapIndex())
* will be moved to the pre-filtering stage.
*
* Any subfilters that cannot be processed entirely with bitmap indexes will be moved to the post-filtering stage.
*/
final Offset offset;
final List<Filter> postFilters = new ArrayList<>();
if (filter == null) {
offset = new NoFilterOffset(0, index.getNumRows(), descending);
} else {
final ColumnSelectorBitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
index.getBitmapFactoryForDimensions(),
index
);
final List<Filter> preFilters = new ArrayList<>();
offset = new BitmapOffset(selector.getBitmapFactory(), filter.getBitmapIndex(selector), descending);
if (filter instanceof AndFilter) {
// If we get an AndFilter, we can split the subfilters across both filtering stages
for (Filter subfilter : ((AndFilter) filter).getFilters()) {
if (subfilter.supportsBitmapIndex(selector)) {
preFilters.add(subfilter);
} else {
postFilters.add(subfilter);
}
}
} else {
// If we get an OrFilter or a single filter, handle the filter in one stage
if (filter.supportsBitmapIndex(selector)) {
preFilters.add(filter);
} else {
postFilters.add(filter);
}
}
if (preFilters.size() == 0) {
offset = new NoFilterOffset(0, index.getNumRows(), descending);
} else {
List<ImmutableBitmap> bitmaps = Lists.newArrayList();
for (Filter prefilter : preFilters) {
bitmaps.add(prefilter.getBitmapIndex(selector));
}
offset = new BitmapOffset(
selector.getBitmapFactory(),
selector.getBitmapFactory().intersection(bitmaps),
descending
);
}
}
final Filter postFilter;
if (postFilters.size() == 0) {
postFilter = null;
} else if (postFilters.size() == 1) {
postFilter = postFilters.get(0);
} else {
postFilter = new AndFilter(postFilters);
}
return Sequences.filter(
@ -233,12 +306,25 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
offset,
minDataTimestamp,
maxDataTimestamp,
descending
descending,
postFilter,
selector
).build(),
Predicates.<Cursor>notNull()
);
}
private interface CursorAdvancer
{
public void advance();
public void advanceTo(int offset);
public boolean isDone();
public void reset();
}
private static class CursorSequenceBuilder
{
private final ColumnSelector index;
@ -248,6 +334,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
private final long minDataTimestamp;
private final long maxDataTimestamp;
private final boolean descending;
private final Filter postFilter;
private final ColumnSelectorBitmapIndexSelector bitmapIndexSelector;
public CursorSequenceBuilder(
ColumnSelector index,
@ -256,7 +344,9 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
Offset offset,
long minDataTimestamp,
long maxDataTimestamp,
boolean descending
boolean descending,
Filter postFilter,
ColumnSelectorBitmapIndexSelector bitmapIndexSelector
)
{
this.index = index;
@ -266,6 +356,8 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
this.minDataTimestamp = minDataTimestamp;
this.maxDataTimestamp = maxDataTimestamp;
this.descending = descending;
this.postFilter = postFilter;
this.bitmapIndexSelector = bitmapIndexSelector;
}
public Sequence<Cursor> build()
@ -323,48 +415,14 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
maxDataTimestamp < timeEnd
);
return new Cursor()
final Offset initOffset = offset.clone();
final DateTime myBucket = gran.toDateTime(input);
final CursorOffsetHolder cursorOffsetHolder = new CursorOffsetHolder();
abstract class QueryableIndexBaseCursor implements Cursor
{
private final Offset initOffset = offset.clone();
private final DateTime myBucket = gran.toDateTime(input);
private Offset cursorOffset = offset;
@Override
public DateTime getTime()
{
return myBucket;
}
@Override
public void advance()
{
if (Thread.interrupted()) {
throw new QueryInterruptedException(new InterruptedException());
}
cursorOffset.increment();
}
@Override
public void advanceTo(int offset)
{
int count = 0;
while (count < offset && !isDone()) {
advance();
count++;
}
}
@Override
public boolean isDone()
{
return !cursorOffset.withinBounds();
}
@Override
public void reset()
{
cursorOffset = initOffset.clone();
}
Offset cursorOffset;
@Override
public DimensionSelector makeDimensionSelector(
@ -507,6 +565,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
}
}
@Override
public FloatColumnSelector makeFloatColumnSelector(String columnName)
{
@ -579,6 +638,7 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
};
}
@Override
public ObjectColumnSelector makeObjectColumnSelector(String column)
{
@ -730,7 +790,143 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
}
};
}
};
}
if (postFilter == null) {
return new QueryableIndexBaseCursor()
{
{
reset();
}
@Override
public DateTime getTime()
{
return myBucket;
}
@Override
public void advance()
{
if (Thread.interrupted()) {
throw new QueryInterruptedException(new InterruptedException());
}
cursorOffset.increment();
}
@Override
public void advanceTo(int offset)
{
int count = 0;
while (count < offset && !isDone()) {
advance();
count++;
}
}
@Override
public boolean isDone()
{
return !cursorOffset.withinBounds();
}
@Override
public void reset()
{
cursorOffset = initOffset.clone();
cursorOffsetHolder.set(cursorOffset);
}
};
} else {
return new QueryableIndexBaseCursor()
{
CursorOffsetHolderValueMatcherFactory valueMatcherFactory = new CursorOffsetHolderValueMatcherFactory(
this
);
RowOffsetMatcherFactory rowOffsetMatcherFactory = new CursorOffsetHolderRowOffsetMatcherFactory(
cursorOffsetHolder,
descending
);
final ValueMatcher filterMatcher;
{
if (postFilter instanceof BooleanFilter) {
filterMatcher = ((BooleanFilter) postFilter).makeMatcher(
bitmapIndexSelector,
valueMatcherFactory,
rowOffsetMatcherFactory
);
} else {
if (postFilter.supportsBitmapIndex(bitmapIndexSelector)) {
filterMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher(postFilter.getBitmapIndex(
bitmapIndexSelector));
} else {
filterMatcher = postFilter.makeMatcher(valueMatcherFactory);
}
}
}
{
reset();
}
@Override
public DateTime getTime()
{
return myBucket;
}
@Override
public void advance()
{
if (Thread.interrupted()) {
throw new QueryInterruptedException(new InterruptedException());
}
cursorOffset.increment();
while (!isDone()) {
if (Thread.interrupted()) {
throw new QueryInterruptedException(new InterruptedException());
}
if (filterMatcher.matches()) {
return;
} else {
cursorOffset.increment();
}
}
}
@Override
public void advanceTo(int offset)
{
int count = 0;
while (count < offset && !isDone()) {
advance();
count++;
}
}
@Override
public boolean isDone()
{
return !cursorOffset.withinBounds();
}
@Override
public void reset()
{
cursorOffset = initOffset.clone();
cursorOffsetHolder.set(cursorOffset);
if (!isDone()) {
if (filterMatcher.matches()) {
return;
} else {
advance();
}
}
}
};
}
}
}
),
@ -760,6 +956,167 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
}
}
public static class CursorOffsetHolder
{
Offset currOffset = null;
public Offset get()
{
return currOffset;
}
public void set(Offset currOffset)
{
this.currOffset = currOffset;
}
}
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 ColumnSelectorFactory cursor;
public CursorOffsetHolderValueMatcherFactory(
ColumnSelectorFactory cursor
)
{
this.cursor = cursor;
}
// Currently unused, except by unit tests, since filters always support bitmap indexes currently.
// This will change when non-String dimensions are added.
@Override
public ValueMatcher makeValueMatcher(String dimension, final Comparable value)
{
final DimensionSelector selector = cursor.makeDimensionSelector(
new DefaultDimensionSpec(dimension, dimension)
);
// if matching against null, rows with size 0 should also match
final boolean matchNull = isComparableNullOrEmpty(value);
final int id = selector.lookupId((String) value);
if (id < 0) {
return new BooleanValueMatcher(false);
} else {
return new ValueMatcher()
{
@Override
public boolean matches()
{
IndexedInts row = selector.getRow();
if (row.size() == 0) {
return matchNull;
}
for (int i = 0; i < row.size(); i++) {
if (row.get(i) == id) {
return true;
}
}
return false;
}
};
}
}
// Currently unused, except by unit tests, since filters always support bitmap indexes currently.
// This will change when non-String dimensions are added.
@Override
public ValueMatcher makeValueMatcher(String dimension, final Predicate predicate)
{
final DimensionSelector selector = cursor.makeDimensionSelector(
new DefaultDimensionSpec(dimension, dimension)
);
return new ValueMatcher()
{
final boolean matchNull = predicate.apply(null);
@Override
public boolean matches()
{
IndexedInts row = selector.getRow();
if (row.size() == 0) {
return matchNull;
}
for (int i = 0; i < row.size(); i++) {
if (predicate.apply(selector.lookupName(row.get(i)))) {
return true;
}
}
return false;
}
};
}
}
private static class CursorOffsetHolderRowOffsetMatcherFactory implements RowOffsetMatcherFactory
{
private final CursorOffsetHolder holder;
private final boolean descending;
public CursorOffsetHolderRowOffsetMatcherFactory(CursorOffsetHolder holder, boolean descending)
{
this.holder = holder;
this.descending = descending;
}
// Use an iterator-based implementation, ImmutableBitmap.get(index) works differently for Concise and Roaring.
// ImmutableConciseSet.get(index) is also inefficient, it performs a linear scan on each call
@Override
public ValueMatcher makeRowOffsetMatcher(final ImmutableBitmap rowBitmap) {
final IntIterator iter = descending ?
BitmapOffset.getReverseBitmapOffsetIterator(rowBitmap) :
rowBitmap.iterator();
if(!iter.hasNext()) {
return new BooleanValueMatcher(false);
}
if (descending) {
return new ValueMatcher()
{
int iterOffset = Integer.MAX_VALUE;
@Override
public boolean matches()
{
int currentOffset = holder.get().getOffset();
while (iterOffset > currentOffset && iter.hasNext()) {
iterOffset = iter.next();
}
return iterOffset == currentOffset;
}
};
} else {
return new ValueMatcher()
{
int iterOffset = -1;
@Override
public boolean matches()
{
int currentOffset = holder.get().getOffset();
while (iterOffset < currentOffset && iter.hasNext()) {
iterOffset = iter.next();
}
return iterOffset == currentOffset;
}
};
}
}
}
private abstract static class TimestampCheckingOffset implements Offset
{
protected final Offset baseOffset;

View File

@ -19,19 +19,25 @@
package io.druid.segment.filter;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.BooleanFilter;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import java.util.ArrayList;
import java.util.List;
/**
*/
public class AndFilter implements Filter
public class AndFilter implements BooleanFilter
{
private static final Joiner AND_JOINER = Joiner.on(" && ");
private final List<Filter> filters;
public AndFilter(
@ -71,6 +77,69 @@ public class AndFilter implements Filter
return makeMatcher(matchers);
}
@Override
public ValueMatcher makeMatcher(
BitmapIndexSelector selector,
ValueMatcherFactory valueMatcherFactory,
RowOffsetMatcherFactory rowOffsetMatcherFactory
)
{
final List<ValueMatcher> matchers = new ArrayList<>();
final List<ImmutableBitmap> bitmaps = new ArrayList<>();
for (Filter filter : filters) {
if (filter.supportsBitmapIndex(selector)) {
bitmaps.add(filter.getBitmapIndex(selector));
} else {
ValueMatcher matcher = filter.makeMatcher(valueMatcherFactory);
matchers.add(matcher);
}
}
if (bitmaps.size() > 0) {
ImmutableBitmap combinedBitmap = selector.getBitmapFactory().intersection(bitmaps);
ValueMatcher offsetMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher(combinedBitmap);
matchers.add(0, offsetMatcher);
}
return new ValueMatcher()
{
@Override
public boolean matches()
{
for (ValueMatcher valueMatcher : matchers) {
if (!valueMatcher.matches()) {
return false;
}
}
return true;
}
};
}
@Override
public List<Filter> getFilters()
{
return filters;
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
for (Filter filter : filters) {
if (!filter.supportsBitmapIndex(selector)) {
return false;
}
}
return true;
}
@Override
public String toString()
{
return String.format("(%s)", AND_JOINER.join(filters));
}
private ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers)
{
if (baseMatchers.length == 1) {
@ -91,4 +160,6 @@ public class AndFilter implements Filter
}
};
}
}

View File

@ -25,10 +25,12 @@ import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
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.column.ColumnCapabilities;
import java.util.Comparator;
import java.util.Iterator;
@ -153,6 +155,12 @@ public class BoundFilter implements Filter
);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return selector.getBitmapIndex(boundDimFilter.getDimension()) != null;
}
private boolean doesMatch(String input)
{
if (extractionFn != null) {

View File

@ -25,8 +25,10 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.column.ColumnCapabilities;
/**
*/
@ -34,6 +36,8 @@ public class DimensionPredicateFilter implements Filter
{
private final String dimension;
private final Predicate<String> predicate;
private final String basePredicateString;
private final ExtractionFn extractionFn;
public DimensionPredicateFilter(
final String dimension,
@ -43,6 +47,8 @@ public class DimensionPredicateFilter implements Filter
{
Preconditions.checkNotNull(predicate, "predicate");
this.dimension = Preconditions.checkNotNull(dimension, "dimension");
this.basePredicateString = predicate.toString();
this.extractionFn = extractionFn;
if (extractionFn == null) {
this.predicate = predicate;
@ -69,4 +75,20 @@ public class DimensionPredicateFilter implements Filter
{
return factory.makeValueMatcher(dimension, predicate);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return selector.getBitmapIndex(dimension) != null;
}
@Override
public String toString()
{
if (extractionFn != null) {
return String.format("%s(%s) = %s", extractionFn, dimension, basePredicateString);
} else {
return String.format("%s = %s", dimension, basePredicateString);
}
}
}

View File

@ -23,14 +23,19 @@ import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.common.IAE;
import com.metamx.common.guava.FunctionalIterable;
import io.druid.query.Query;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.BooleanFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.Filter;
import io.druid.segment.column.BitmapIndex;
import io.druid.segment.data.Indexed;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
@ -38,6 +43,8 @@ import java.util.List;
*/
public class Filters
{
private static final String CTX_KEY_USE_FILTER_CNF = "useFilterCNF";
/**
* Convert a list of DimFilters to a list of Filters.
*
@ -149,4 +156,183 @@ public class Filters
}
);
}
public static Filter convertToCNFFromQueryContext(Query query, Filter filter)
{
if (filter == null) {
return null;
}
boolean useCNF = query.getContextBoolean(CTX_KEY_USE_FILTER_CNF, false);
return useCNF ? convertToCNF(filter) : filter;
}
public static Filter convertToCNF(Filter current)
{
current = pushDownNot(current);
current = flatten(current);
current = convertToCNFInternal(current);
current = flatten(current);
return current;
}
// CNF conversion functions were adapted from Apache Hive, see:
// https://github.com/apache/hive/blob/branch-2.0/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
private static Filter pushDownNot(Filter current)
{
if (current instanceof NotFilter) {
Filter child = ((NotFilter) current).getBaseFilter();
if (child instanceof NotFilter) {
return pushDownNot(((NotFilter) child).getBaseFilter());
}
if (child instanceof AndFilter) {
List<Filter> children = Lists.newArrayList();
for (Filter grandChild : ((AndFilter) child).getFilters()) {
children.add(pushDownNot(new NotFilter(grandChild)));
}
return new OrFilter(children);
}
if (child instanceof OrFilter) {
List<Filter> children = Lists.newArrayList();
for (Filter grandChild : ((OrFilter) child).getFilters()) {
children.add(pushDownNot(new NotFilter(grandChild)));
}
return new AndFilter(children);
}
}
if (current instanceof AndFilter) {
List<Filter> children = Lists.newArrayList();
for (Filter child : ((AndFilter) current).getFilters()) {
children.add(pushDownNot(child));
}
return new AndFilter(children);
}
if (current instanceof OrFilter) {
List<Filter> children = Lists.newArrayList();
for (Filter child : ((OrFilter) current).getFilters()) {
children.add(pushDownNot(child));
}
return new OrFilter(children);
}
return current;
}
// CNF conversion functions were adapted from Apache Hive, see:
// https://github.com/apache/hive/blob/branch-2.0/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
private static Filter convertToCNFInternal(Filter current)
{
if (current instanceof NotFilter) {
return new NotFilter(convertToCNFInternal(((NotFilter) current).getBaseFilter()));
}
if (current instanceof AndFilter) {
List<Filter> children = Lists.newArrayList();
for (Filter child : ((AndFilter) current).getFilters()) {
children.add(convertToCNFInternal(child));
}
return new AndFilter(children);
}
if (current instanceof OrFilter) {
// a list of leaves that weren't under AND expressions
List<Filter> nonAndList = new ArrayList<Filter>();
// a list of AND expressions that we need to distribute
List<Filter> andList = new ArrayList<Filter>();
for (Filter child : ((OrFilter) current).getFilters()) {
if (child instanceof AndFilter) {
andList.add(child);
} else if (child instanceof OrFilter) {
// pull apart the kids of the OR expression
for (Filter grandChild : ((OrFilter) child).getFilters()) {
nonAndList.add(grandChild);
}
} else {
nonAndList.add(child);
}
}
if (!andList.isEmpty()) {
List<Filter> result = Lists.newArrayList();
generateAllCombinations(result, andList, nonAndList);
return new AndFilter(result);
}
}
return current;
}
// CNF conversion functions were adapted from Apache Hive, see:
// https://github.com/apache/hive/blob/branch-2.0/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
private static Filter flatten(Filter root) {
if (root instanceof BooleanFilter) {
List<Filter> children = new ArrayList<>();
children.addAll(((BooleanFilter) root).getFilters());
// iterate through the index, so that if we add more children,
// they don't get re-visited
for (int i = 0; i < children.size(); ++i) {
Filter child = flatten(children.get(i));
// do we need to flatten?
if (child.getClass() == root.getClass() && !(child instanceof NotFilter)) {
boolean first = true;
List<Filter> grandKids = ((BooleanFilter)child).getFilters();
for (Filter grandkid : grandKids) {
// for the first grandkid replace the original parent
if (first) {
first = false;
children.set(i, grandkid);
} else {
children.add(++i, grandkid);
}
}
} else {
children.set(i, child);
}
}
// if we have a singleton AND or OR, just return the child
if (children.size() == 1 && (root instanceof AndFilter || root instanceof OrFilter)) {
return children.get(0);
}
if (root instanceof AndFilter) {
return new AndFilter(children);
} else if (root instanceof OrFilter) {
return new OrFilter(children);
}
}
return root;
}
// CNF conversion functions were adapted from Apache Hive, see:
// https://github.com/apache/hive/blob/branch-2.0/storage-api/src/java/org/apache/hadoop/hive/ql/io/sarg/SearchArgumentImpl.java
private static void generateAllCombinations(
List<Filter> result,
List<Filter> andList,
List<Filter> nonAndList
)
{
List<Filter> children = ((AndFilter) andList.get(0)).getFilters();
if (result.isEmpty()) {
for (Filter child : children) {
List<Filter> a = Lists.newArrayList(nonAndList);
a.add(child);
result.add(new OrFilter(a));
}
} else {
List<Filter> work = new ArrayList<>(result);
result.clear();
for (Filter child : children) {
for (Filter or : work) {
List<Filter> a = Lists.newArrayList((((OrFilter) or).getFilters()));
a.add(child);
result.add(new OrFilter(a));
}
}
}
if (andList.size() > 1) {
generateAllCombinations(
result, andList.subList(1, andList.size()),
nonAndList
);
}
}
}

View File

@ -27,8 +27,10 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.column.ColumnCapabilities;
import java.util.Set;
@ -97,4 +99,10 @@ public class InFilter implements Filter
}
);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return selector.getBitmapIndex(dimension) != null;
}
}

View File

@ -24,8 +24,10 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.JavaScriptDimFilter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.column.ColumnCapabilities;
import org.mozilla.javascript.Context;
public class JavaScriptFilter implements Filter
@ -69,4 +71,10 @@ public class JavaScriptFilter implements Filter
// suboptimal, since we need create one context per call to predicate.apply()
return factory.makeValueMatcher(dimension, predicate);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return selector.getBitmapIndex(dimension) != null;
}
}

View File

@ -22,6 +22,7 @@ package io.druid.segment.filter;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
@ -61,4 +62,15 @@ public class NotFilter implements Filter
}
};
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return baseFilter.supportsBitmapIndex(selector);
}
public Filter getBaseFilter()
{
return baseFilter;
}
}

View File

@ -19,19 +19,25 @@
package io.druid.segment.filter;
import com.google.common.base.Joiner;
import com.google.common.collect.Lists;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.BooleanFilter;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import java.util.ArrayList;
import java.util.List;
/**
*/
public class OrFilter implements Filter
public class OrFilter implements BooleanFilter
{
private static final Joiner OR_JOINER = Joiner.on(" || ");
private final List<Filter> filters;
public OrFilter(
@ -71,6 +77,47 @@ public class OrFilter implements Filter
return makeMatcher(matchers);
}
@Override
public ValueMatcher makeMatcher(
BitmapIndexSelector selector,
ValueMatcherFactory valueMatcherFactory,
RowOffsetMatcherFactory rowOffsetMatcherFactory
)
{
final List<ValueMatcher> matchers = new ArrayList<>();
final List<ImmutableBitmap> bitmaps = new ArrayList<>();
for (Filter filter : filters) {
if (filter.supportsBitmapIndex(selector)) {
bitmaps.add(filter.getBitmapIndex(selector));
} else {
ValueMatcher matcher = filter.makeMatcher(valueMatcherFactory);
matchers.add(matcher);
}
}
if (bitmaps.size() > 0) {
ImmutableBitmap combinedBitmap = selector.getBitmapFactory().union(bitmaps);
ValueMatcher offsetMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher(combinedBitmap);
matchers.add(0, offsetMatcher);
}
return new ValueMatcher()
{
@Override
public boolean matches()
{
for (ValueMatcher valueMatcher : matchers) {
if (valueMatcher.matches()) {
return true;
}
}
return false;
}
};
}
private ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers){
if (baseMatchers.length == 1) {
return baseMatchers[0];
@ -91,4 +138,25 @@ public class OrFilter implements Filter
};
}
@Override
public List<Filter> getFilters()
{
return filters;
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
for (Filter filter : filters) {
if(!filter.supportsBitmapIndex(selector)) {
return false;
}
}
return true;
}
public String toString()
{
return String.format("(%s)", OR_JOINER.join(filters));
}
}

View File

@ -29,7 +29,7 @@ import java.util.regex.Pattern;
public class RegexFilter extends DimensionPredicateFilter
{
public RegexFilter(
String dimension,
final String dimension,
final Pattern pattern,
final ExtractionFn extractionFn
)
@ -43,6 +43,14 @@ public class RegexFilter extends DimensionPredicateFilter
{
return (input != null) && pattern.matcher(input).find();
}
@Override
public String toString()
{
return "RegexFilter{" +
"pattern='" + pattern + '\'' +
'}';
}
},
extractionFn
);

View File

@ -33,7 +33,7 @@ public class SearchQueryFilter extends DimensionPredicateFilter
{
@JsonCreator
public SearchQueryFilter(
@JsonProperty("dimension") String dimension,
@JsonProperty("dimension") final String dimension,
@JsonProperty("query") final SearchQuerySpec query,
@JsonProperty("extractionFn") final ExtractionFn extractionFn
)
@ -47,6 +47,14 @@ public class SearchQueryFilter extends DimensionPredicateFilter
{
return query.accept(input);
}
@Override
public String toString()
{
return "SearchQueryFilter{" +
", query=" + query +
'}';
}
},
extractionFn
);

View File

@ -22,8 +22,10 @@ package io.druid.segment.filter;
import com.metamx.collections.bitmap.ImmutableBitmap;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.column.ColumnCapabilities;
/**
*/
@ -52,4 +54,16 @@ public class SelectorFilter implements Filter
{
return factory.makeValueMatcher(dimension, value);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return selector.getBitmapIndex(dimension) != null;
}
@Override
public String toString()
{
return String.format("%s = %s", dimension, value);
}
}

View File

@ -24,12 +24,12 @@ import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.collections.spatial.search.Bound;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.RowOffsetMatcherFactory;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.column.ColumnCapabilities;
import io.druid.segment.incremental.SpatialDimensionRowTransformer;
import java.util.Arrays;
/**
*/
public class SpatialFilter implements Filter
@ -73,4 +73,10 @@ public class SpatialFilter implements Filter
}
);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return selector.getBitmapIndex(dimension) != null;
}
}

View File

@ -252,7 +252,7 @@ public abstract class BaseFilterTest
return constructors;
}
private DimFilter maybeOptimize(final DimFilter dimFilter)
protected DimFilter maybeOptimize(final DimFilter dimFilter)
{
if (dimFilter == null) {
return null;
@ -260,7 +260,7 @@ public abstract class BaseFilterTest
return optimize ? dimFilter.optimize() : dimFilter;
}
private Sequence<Cursor> makeCursorSequence(final Filter filter)
protected Sequence<Cursor> makeCursorSequence(final Filter filter)
{
final Sequence<Cursor> cursors = adapter.makeCursors(
filter,

View File

@ -0,0 +1,718 @@
/*
* 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.base.Strings;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
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.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.js.JavaScriptConfig;
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.extraction.ExtractionFn;
import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.filter.AndDimFilter;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.DimFilter;
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;
import org.junit.Test;
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;
import java.util.Objects;
@RunWith(Parameterized.class)
public class FilterPartitionTest extends BaseFilterTest
{
private class NoBitmapSelectorFilter extends SelectorFilter
{
public NoBitmapSelectorFilter(
String dimension,
String value
)
{
super(dimension, value);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return false;
}
}
private class NoBitmapDimensionPredicateFilter extends DimensionPredicateFilter
{
public NoBitmapDimensionPredicateFilter(
final String dimension,
final Predicate<String> predicate,
final ExtractionFn extractionFn
)
{
super(dimension, predicate, extractionFn);
}
@Override
public boolean supportsBitmapIndex(BitmapIndexSelector selector)
{
return false;
}
}
private class NoBitmapSelectorDimFilter extends SelectorDimFilter
{
public NoBitmapSelectorDimFilter(
String dimension,
String value,
ExtractionFn extractionFn
)
{
super(dimension, value, extractionFn);
}
@Override
public Filter toFilter()
{
ExtractionFn extractionFn = getExtractionFn();
String dimension = getDimension();
String value = getValue();
if (extractionFn == null) {
return new NoBitmapSelectorFilter(dimension, value);
} else {
final String valueOrNull = Strings.emptyToNull(value);
final Predicate<String> predicate = new Predicate<String>()
{
@Override
public boolean apply(String input)
{
return Objects.equals(valueOrNull, input);
}
};
return new NoBitmapDimensionPredicateFilter(dimension, predicate, extractionFn);
}
}
}
private static String JS_FN = "function(str) { return 'super-' + str; }";
private static ExtractionFn JS_EXTRACTION_FN = new JavaScriptExtractionFn(JS_FN, false, JavaScriptConfig.getDefault());
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(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim0", "dim1", "dim2", "dim3")),
null,
null
)
)
);
private static final List<InputRow> ROWS = ImmutableList.of(
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "0", "dim1", "", "dim2", ImmutableList.of("a", "b"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "1", "dim1", "10", "dim2", ImmutableList.of())),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "2", "dim1", "2", "dim2", ImmutableList.of(""))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "3", "dim1", "1", "dim2", ImmutableList.of("a"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "4", "dim1", "def", "dim2", ImmutableList.of("c"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "5", "dim1", "abc")),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "6", "dim1", "B453B411", "dim2", ImmutableList.of("c", "d", "e"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "7", "dim1", "HELLO", "dim2", ImmutableList.of("foo"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "8", "dim1", "abc", "dim2", ImmutableList.of("bar"))),
PARSER.parse(ImmutableMap.<String, Object>of("dim0", "9", "dim1", "1", "dim2", ImmutableList.of("foo", "bar")))
);
public FilterPartitionTest(
String testName,
IndexBuilder indexBuilder,
Function<IndexBuilder, Pair<StorageAdapter, Closeable>> finisher,
boolean optimize
)
{
super(testName, ROWS, indexBuilder, finisher, optimize);
}
@AfterClass
public static void tearDown() throws Exception
{
BaseFilterTest.tearDown(FilterPartitionTest.class.getName());
}
@Test
public void testSinglePreFilterWithNulls()
{
assertFilterMatches(new SelectorDimFilter("dim1", null, null), ImmutableList.of("0"));
assertFilterMatches(new SelectorDimFilter("dim1", "", null), ImmutableList.of("0"));
assertFilterMatches(new SelectorDimFilter("dim1", "10", null), ImmutableList.of("1"));
assertFilterMatches(new SelectorDimFilter("dim1", "2", null), ImmutableList.of("2"));
assertFilterMatches(new SelectorDimFilter("dim1", "1", null), ImmutableList.of("3", "9"));
assertFilterMatches(new SelectorDimFilter("dim1", "def", null), ImmutableList.of("4"));
assertFilterMatches(new SelectorDimFilter("dim1", "abc", null), ImmutableList.of("5", "8"));
assertFilterMatches(new SelectorDimFilter("dim1", "ab", null), ImmutableList.<String>of());
}
@Test
public void testSinglePostFilterWithNulls()
{
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", null, null), ImmutableList.of("0"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "", null), ImmutableList.of("0"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "10", null), ImmutableList.of("1"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "2", null), ImmutableList.of("2"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "1", null), ImmutableList.of("3", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "def", null), ImmutableList.of("4"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "abc", null), ImmutableList.of("5", "8"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "ab", null), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-null", JS_EXTRACTION_FN), ImmutableList.of("0"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-null", JS_EXTRACTION_FN), ImmutableList.of("0"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-10", JS_EXTRACTION_FN), ImmutableList.of("1"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-2", JS_EXTRACTION_FN), ImmutableList.of("2"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-1", JS_EXTRACTION_FN), ImmutableList.of("3", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-def", JS_EXTRACTION_FN), ImmutableList.of("4"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-abc", JS_EXTRACTION_FN), ImmutableList.of("5", "8"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim1", "super-ab", JS_EXTRACTION_FN), ImmutableList.<String>of());
}
@Test
public void testBasicPreAndPostFilterWithNulls()
{
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim2", "a", null),
new NoBitmapSelectorDimFilter("dim1", null, null)
)),
ImmutableList.of("0")
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "10", null),
new NoBitmapSelectorDimFilter("dim2", null, null)
)),
ImmutableList.of("1")
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "1", null),
new NoBitmapSelectorDimFilter("dim2", "foo", null)
)),
ImmutableList.of("9")
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "HELLO", null),
new NoBitmapSelectorDimFilter("dim2", "bar", null)
)),
ImmutableList.<String>of()
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim2", "bar", null),
new SelectorDimFilter("dim1", "NOT_A_VALUE", null)
)),
ImmutableList.<String>of()
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim2", "super-a", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim1", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("0")
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "super-10", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("1")
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "super-2", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("2")
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "super-1", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-foo", JS_EXTRACTION_FN)
)),
ImmutableList.of("9")
);
assertFilterMatches(
new AndDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "super-HELLO", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-bar", JS_EXTRACTION_FN)
)),
ImmutableList.<String>of()
);
}
@Test
public void testOrPostFilterWithNulls()
{
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim2", "a", null),
new NoBitmapSelectorDimFilter("dim1", null, null)
)),
ImmutableList.of("0", "3")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "abc", null),
new NoBitmapSelectorDimFilter("dim2", null, null)
)),
ImmutableList.of("1", "2", "5", "8")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "2", null),
new NoBitmapSelectorDimFilter("dim2", null, null)
)),
ImmutableList.of("1", "2", "5")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "INVALID_VALUE", null),
new NoBitmapSelectorDimFilter("dim2", "foo", null)
)),
ImmutableList.of("7", "9")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "HELLO", null),
new NoBitmapSelectorDimFilter("dim2", "bar", null)
)),
ImmutableList.<String>of("7", "8", "9")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "HELLO", null),
new SelectorDimFilter("dim2", "NOT_A_VALUE", null)
)),
ImmutableList.<String>of("7")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "INVALID", null),
new SelectorDimFilter("dim2", "NOT_A_VALUE", null)
)),
ImmutableList.<String>of()
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim2", "super-a", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim1", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("0", "3")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "super-abc", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("1", "2", "5", "8")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "super-2", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("1", "2", "5")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "INVALID_VALUE", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-foo", JS_EXTRACTION_FN)
)),
ImmutableList.of("7", "9")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim1", "super-HELLO", JS_EXTRACTION_FN),
new NoBitmapSelectorDimFilter("dim2", "super-bar", JS_EXTRACTION_FN)
)),
ImmutableList.<String>of("7", "8", "9")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "super-HELLO", JS_EXTRACTION_FN),
new SelectorDimFilter("dim2", "NOT_A_VALUE", null)
)),
ImmutableList.<String>of("7")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "INVALID", JS_EXTRACTION_FN),
new SelectorDimFilter("dim2", "NOT_A_VALUE", JS_EXTRACTION_FN)
)),
ImmutableList.<String>of()
);
}
@Test
public void testMissingColumnSpecifiedInDimensionList()
{
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", null, null), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "", null), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "a", null), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "b", null), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "c", null), ImmutableList.<String>of());
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "abc", null),
new SelectorDimFilter("dim3", "NOTHERE", null)
)),
ImmutableList.<String>of("5", "8")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "abc", null),
new SelectorDimFilter("dim3", null, null)
)),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
);
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "super-null", JS_EXTRACTION_FN),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "super-null", JS_EXTRACTION_FN),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "a", JS_EXTRACTION_FN), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "b", JS_EXTRACTION_FN), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim3", "c", JS_EXTRACTION_FN), ImmutableList.<String>of());
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "super-abc", JS_EXTRACTION_FN),
new SelectorDimFilter("dim3", "NOTHERE", JS_EXTRACTION_FN)
)),
ImmutableList.<String>of("5", "8")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "abc", JS_EXTRACTION_FN),
new SelectorDimFilter("dim3", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
);
}
@Test
public void testMissingColumnNotSpecifiedInDimensionList()
{
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", null, null), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "", null), ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "a", null), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "b", null), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "c", null), ImmutableList.<String>of());
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "abc", null),
new SelectorDimFilter("dim4", null, null)
)),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim4", null, null),
new SelectorDimFilter("dim1", "abc", null)
)),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
);
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "super-null", JS_EXTRACTION_FN),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "super-null", JS_EXTRACTION_FN),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9"));
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "a", JS_EXTRACTION_FN), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "b", JS_EXTRACTION_FN), ImmutableList.<String>of());
assertFilterMatches(new NoBitmapSelectorDimFilter("dim4", "c", JS_EXTRACTION_FN), ImmutableList.<String>of());
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "super-abc", JS_EXTRACTION_FN),
new SelectorDimFilter("dim4", "super-null", JS_EXTRACTION_FN)
)),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
);
assertFilterMatches(
new OrDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim4", "super-null", JS_EXTRACTION_FN),
new SelectorDimFilter("dim1", "super-abc", JS_EXTRACTION_FN)
)),
ImmutableList.of("0", "1", "2", "3", "4", "5", "6", "7", "8", "9")
);
}
@Test
public void testDistributeOrCNF()
{
DimFilter dimFilter1 = new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim0", "6", null),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "def", null),
new SelectorDimFilter("dim2", "c", null)
)
))
);
Filter filter1 = dimFilter1.toFilter();
Filter filter1CNF = Filters.convertToCNF(filter1);
Assert.assertEquals(AndFilter.class, filter1CNF.getClass());
Assert.assertEquals(2, ((AndFilter) filter1CNF).getFilters().size());
assertFilterMatches(
dimFilter1,
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),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "HELLO", null),
new SelectorDimFilter("dim2", "foo", null)
)
))
);
assertFilterMatches(
dimFilter2,
ImmutableList.of("2", "3", "7")
);
assertFilterMatchesCNF(
dimFilter2,
ImmutableList.of("2", "3", "7")
);
DimFilter dimFilter3 = new OrDimFilter(Arrays.<DimFilter>asList(
dimFilter1,
dimFilter2,
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "1", null),
new SelectorDimFilter("dim2", "foo", null)
)
))
);
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()
{
DimFilter dimFilter1 = new OrDimFilter(Arrays.<DimFilter>asList(
new SelectorDimFilter("dim0", "super-6", JS_EXTRACTION_FN),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "super-def", JS_EXTRACTION_FN),
new SelectorDimFilter("dim2", "super-c", JS_EXTRACTION_FN)
)
))
);
Filter filter1 = dimFilter1.toFilter();
Filter filter1CNF = Filters.convertToCNF(filter1);
Assert.assertEquals(AndFilter.class, filter1CNF.getClass());
Assert.assertEquals(2, ((AndFilter) filter1CNF).getFilters().size());
assertFilterMatches(
dimFilter1,
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),
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "super-HELLO", JS_EXTRACTION_FN),
new SelectorDimFilter("dim2", "super-foo", JS_EXTRACTION_FN)
)
))
);
assertFilterMatches(
dimFilter2,
ImmutableList.of("2", "3", "7")
);
assertFilterMatchesCNF(
dimFilter2,
ImmutableList.of("2", "3", "7")
);
DimFilter dimFilter3 = new OrDimFilter(Arrays.<DimFilter>asList(
dimFilter1,
dimFilter2,
new AndDimFilter(Arrays.<DimFilter>asList(
new NoBitmapSelectorDimFilter("dim1", "super-1", JS_EXTRACTION_FN),
new SelectorDimFilter("dim2", "super-foo", JS_EXTRACTION_FN)
)
))
);
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")
);
}
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

@ -20,10 +20,8 @@
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;
@ -36,9 +34,7 @@ import io.druid.js.JavaScriptConfig;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.JavaScriptExtractionFn;
import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.filter.BoundDimFilter;
import io.druid.query.filter.DimFilter;
import io.druid.query.filter.Filter;
import io.druid.query.filter.InDimFilter;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.lookup.LookupExtractor;
@ -54,7 +50,6 @@ import org.junit.runners.Parameterized;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;

View File

@ -33,7 +33,6 @@ 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.query.filter.SearchQueryDimFilter;
import io.druid.query.search.search.ContainsSearchQuerySpec;
import io.druid.query.search.search.SearchQuerySpec;

View File

@ -33,7 +33,6 @@ 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.OrDimFilter;
import io.druid.query.filter.SelectorDimFilter;
import io.druid.query.lookup.LookupExtractionFn;
import io.druid.query.lookup.LookupExtractor;