DimensionPredicateFilter: Skip unnecessary dictionary lookup. (#2850)

This commit is contained in:
Gian Merlino 2016-04-18 12:38:25 -07:00 committed by Xavier Léauté
parent f5822faca3
commit 7c0b1dde3a
2 changed files with 248 additions and 50 deletions

View File

@ -0,0 +1,187 @@
/*
* 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.base.Predicate;
import com.google.common.collect.FluentIterable;
import com.metamx.collections.bitmap.BitmapFactory;
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.segment.column.BitmapIndex;
import io.druid.segment.data.BitmapSerdeFactory;
import io.druid.segment.data.GenericIndexed;
import io.druid.segment.data.Indexed;
import io.druid.segment.data.RoaringBitmapSerdeFactory;
import io.druid.segment.filter.DimensionPredicateFilter;
import io.druid.segment.serde.BitmapIndexColumnPartSupplier;
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 DimensionPredicateFilterBenchmark
{
private static final int START_INT = 1_000_000_000;
private static final DimensionPredicateFilter IS_EVEN = new DimensionPredicateFilter(
"foo",
new Predicate<String>()
{
@Override
public boolean apply(String input)
{
if (input == null) {
return false;
}
return Integer.parseInt(input) % 2 == 0;
}
},
null
);
// cardinality, the dictionary will contain integers starting from START_INT
@Param({"1000", "100000", "1000000"})
int cardinality;
// selector will contain a cardinality number of bitmaps; each one contains a single int: 0
BitmapIndexSelector selector;
@Setup
public void setup() throws IOException
{
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);
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 matchIsEven()
{
final ImmutableBitmap bitmapIndex = IS_EVEN.getBitmapIndex(selector);
Preconditions.checkState(bitmapIndex.size() == cardinality / 2);
}
private List<Integer> generateInts()
{
final List<Integer> ints = new ArrayList<>(cardinality);
for (int i = 0; i < cardinality; i++) {
ints.add(START_INT + i);
}
return ints;
}
}

View File

@ -19,36 +19,45 @@
package io.druid.segment.filter;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.metamx.collections.bitmap.ImmutableBitmap;
import com.metamx.common.guava.FunctionalIterable;
import io.druid.query.extraction.ExtractionFn;
import io.druid.query.filter.BitmapIndexSelector;
import io.druid.query.filter.Filter;
import io.druid.query.filter.ValueMatcher;
import io.druid.query.filter.ValueMatcherFactory;
import io.druid.segment.column.BitmapIndex;
import io.druid.segment.data.Indexed;
import javax.annotation.Nullable;
import java.util.Iterator;
/**
*/
class DimensionPredicateFilter implements Filter
public class DimensionPredicateFilter implements Filter
{
private final String dimension;
private final Predicate<String> predicate;
private final ExtractionFn extractionFn;
public DimensionPredicateFilter(
String dimension,
Predicate<String> predicate,
ExtractionFn extractionFn
final String dimension,
final Predicate<String> predicate,
final ExtractionFn extractionFn
)
{
this.dimension = dimension;
this.predicate = predicate;
this.extractionFn = extractionFn;
if (predicate == null) {
this.predicate = null;
} else {
this.predicate = extractionFn == null ? predicate : new Predicate<String>()
{
@Override
public boolean apply(String input)
{
return predicate.apply(extractionFn.apply(input));
}
};
}
}
@Override
@ -57,10 +66,9 @@ class DimensionPredicateFilter implements Filter
if (predicate == null) {
return selector.getBitmapFactory().makeEmptyImmutableBitmap();
}
Indexed<String> dimValues = selector.getDimensionValues(dimension);
final Indexed<String> dimValues = selector.getDimensionValues(dimension);
if (dimValues == null || dimValues.size() == 0) {
boolean needsComplement = predicate.apply(extractionFn == null ? null : extractionFn.apply(null));
if (needsComplement) {
if (predicate.apply(null)) {
return selector.getBitmapFactory().complement(
selector.getBitmapFactory().makeEmptyImmutableBitmap(),
selector.getNumRows()
@ -71,49 +79,52 @@ class DimensionPredicateFilter implements Filter
}
final BitmapIndex bitmapIndex = selector.getBitmapIndex(dimension);
return selector.getBitmapFactory().union(
FunctionalIterable.create(dimValues)
.filter(
extractionFn == null ?
predicate
:
new Predicate<String>()
{
@Override
public boolean apply(@Nullable String input)
{
return predicate.apply(extractionFn.apply(input));
}
}
)
.transform(
new Function<String, ImmutableBitmap>()
{
@Override
public ImmutableBitmap apply(@Nullable String input)
{
return selector.getBitmapIndex(dimension, input);
}
}
)
new Iterable<ImmutableBitmap>()
{
@Override
public Iterator<ImmutableBitmap> iterator()
{
return new Iterator<ImmutableBitmap>()
{
int currIndex = 0;
@Override
public boolean hasNext()
{
return currIndex < bitmapIndex.getCardinality();
}
@Override
public ImmutableBitmap next()
{
while (currIndex < bitmapIndex.getCardinality() && !predicate.apply(dimValues.get(currIndex))) {
currIndex++;
}
if (currIndex == bitmapIndex.getCardinality()) {
return bitmapIndex.getBitmapFactory().makeEmptyImmutableBitmap();
}
return bitmapIndex.getBitmap(currIndex++);
}
@Override
public void remove()
{
throw new UnsupportedOperationException();
}
};
}
}
);
}
@Override
public ValueMatcher makeMatcher(ValueMatcherFactory factory)
{
if (extractionFn == null) {
return factory.makeValueMatcher(dimension, predicate);
} else {
Predicate extractingPredicate = new Predicate()
{
@Override
public boolean apply(@Nullable Object input)
{
return predicate.apply(extractionFn.apply(input));
}
};
return factory.makeValueMatcher(dimension, extractingPredicate);
}
return factory.makeValueMatcher(dimension, predicate);
}
}