mirror of https://github.com/apache/druid.git
Lazy-ify ValueMatcher BitSet optimization for string dimensions. (#5403)
* Lazy-ify ValueMatcher BitSet optimization for string dimensions. The idea is that if the prior evaluated filters are decently selective, such that they mean we won't see all possible values of the later filters, then the eager version of the optimization is too wasteful. This involves checking an extra bitset, but the overhead is small even if the lazy-ification is useless. * Remove import. * Minor transformation
This commit is contained in:
parent
d1cdcd4895
commit
78fd27cdb2
|
@ -20,9 +20,9 @@
|
|||
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.ImmutableList;
|
||||
import com.google.common.io.Files;
|
||||
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
||||
|
@ -242,12 +242,7 @@ public class FilterPartitionBenchmark
|
|||
{
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, null);
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -258,11 +253,7 @@ public class FilterPartitionBenchmark
|
|||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, null);
|
||||
|
||||
Sequence<List<Long>> longListSeq = readCursorsLong(cursors, blackhole);
|
||||
List<Long> strings = longListSeq.limit(1).toList().get(0);
|
||||
for (Long st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursorsLong(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -273,11 +264,7 @@ public class FilterPartitionBenchmark
|
|||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, timeFilterNone);
|
||||
|
||||
Sequence<List<Long>> longListSeq = readCursorsLong(cursors, blackhole);
|
||||
List<Long> strings = longListSeq.limit(1).toList().get(0);
|
||||
for (Long st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursorsLong(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -288,11 +275,7 @@ public class FilterPartitionBenchmark
|
|||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, timeFilterHalf);
|
||||
|
||||
Sequence<List<Long>> longListSeq = readCursorsLong(cursors, blackhole);
|
||||
List<Long> strings = longListSeq.limit(1).toList().get(0);
|
||||
for (Long st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursorsLong(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -303,11 +286,7 @@ public class FilterPartitionBenchmark
|
|||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, timeFilterAll);
|
||||
|
||||
Sequence<List<Long>> longListSeq = readCursorsLong(cursors, blackhole);
|
||||
List<Long> strings = longListSeq.limit(1).toList().get(0);
|
||||
for (Long st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursorsLong(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -319,12 +298,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, filter);
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -336,12 +310,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, filter);
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -353,12 +322,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, filter);
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -370,12 +334,24 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, filter);
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
@Benchmark
|
||||
@BenchmarkMode(Mode.AverageTime)
|
||||
@OutputTimeUnit(TimeUnit.MICROSECONDS)
|
||||
public void readAndFilter(Blackhole blackhole)
|
||||
{
|
||||
Filter andFilter = new AndFilter(
|
||||
ImmutableList.of(
|
||||
new SelectorFilter("dimUniform", "199"),
|
||||
new NoBitmapSelectorDimFilter("dimUniform", "super-199", JS_EXTRACTION_FN).toFilter()
|
||||
)
|
||||
);
|
||||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, andFilter);
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -389,12 +365,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, orFilter);
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -408,12 +379,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, Filters.convertToCNF(orFilter));
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -450,12 +416,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, dimFilter3.toFilter());
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
@Benchmark
|
||||
|
@ -492,12 +453,7 @@ public class FilterPartitionBenchmark
|
|||
|
||||
StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
|
||||
Sequence<Cursor> cursors = makeCursors(sa, Filters.convertToCNF(dimFilter3.toFilter()));
|
||||
|
||||
Sequence<List<String>> stringListSeq = readCursors(cursors, blackhole);
|
||||
List<String> strings = stringListSeq.limit(1).toList().get(0);
|
||||
for (String st : strings) {
|
||||
blackhole.consume(st);
|
||||
}
|
||||
readCursors(cursors, blackhole);
|
||||
}
|
||||
|
||||
private Sequence<Cursor> makeCursors(StorageAdapter sa, Filter filter)
|
||||
|
@ -505,55 +461,46 @@ public class FilterPartitionBenchmark
|
|||
return sa.makeCursors(filter, schemaInfo.getDataInterval(), VirtualColumns.EMPTY, Granularities.ALL, false, null);
|
||||
}
|
||||
|
||||
private Sequence<List<String>> readCursors(Sequence<Cursor> cursors, final Blackhole blackhole)
|
||||
private void readCursors(Sequence<Cursor> cursors, Blackhole blackhole)
|
||||
{
|
||||
return Sequences.map(
|
||||
final Sequence<Void> voids = 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.getColumnSelectorFactory().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();
|
||||
input -> {
|
||||
List<DimensionSelector> selectors = new ArrayList<>();
|
||||
selectors.add(
|
||||
input.getColumnSelectorFactory().makeDimensionSelector(new DefaultDimensionSpec("dimSequential", null))
|
||||
);
|
||||
while (!input.isDone()) {
|
||||
for (DimensionSelector selector : selectors) {
|
||||
IndexedInts row = selector.getRow();
|
||||
blackhole.consume(selector.lookupName(row.get(0)));
|
||||
}
|
||||
return strings;
|
||||
input.advance();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
);
|
||||
|
||||
blackhole.consume(voids.toList());
|
||||
}
|
||||
|
||||
private Sequence<List<Long>> readCursorsLong(Sequence<Cursor> cursors, final Blackhole blackhole)
|
||||
private void readCursorsLong(Sequence<Cursor> cursors, final Blackhole blackhole)
|
||||
{
|
||||
return Sequences.map(
|
||||
final Sequence<Void> voids = Sequences.map(
|
||||
cursors,
|
||||
new Function<Cursor, List<Long>>()
|
||||
{
|
||||
@Override
|
||||
public List<Long> apply(Cursor input)
|
||||
{
|
||||
List<Long> longvals = new ArrayList<Long>();
|
||||
BaseLongColumnValueSelector selector = input.getColumnSelectorFactory().makeColumnValueSelector("sumLongSequential");
|
||||
while (!input.isDone()) {
|
||||
long rowval = selector.getLong();
|
||||
blackhole.consume(rowval);
|
||||
input.advance();
|
||||
}
|
||||
return longvals;
|
||||
input -> {
|
||||
BaseLongColumnValueSelector selector = input.getColumnSelectorFactory()
|
||||
.makeColumnValueSelector("sumLongSequential");
|
||||
while (!input.isDone()) {
|
||||
long rowval = selector.getLong();
|
||||
blackhole.consume(rowval);
|
||||
input.advance();
|
||||
}
|
||||
return null;
|
||||
}
|
||||
);
|
||||
|
||||
blackhole.consume(voids.toList());
|
||||
}
|
||||
|
||||
private static class NoBitmapSelectorFilter extends SelectorFilter
|
||||
|
|
|
@ -170,8 +170,11 @@ public final class DimensionSelectorUtils
|
|||
Predicate<String> predicate
|
||||
)
|
||||
{
|
||||
final BitSet predicateMatchingValueIds = makePredicateMatchingSet(selector, predicate);
|
||||
final BitSet checkedIds = new BitSet(selector.getValueCardinality());
|
||||
final BitSet matchingIds = new BitSet(selector.getValueCardinality());
|
||||
final boolean matchNull = predicate.apply(null);
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
|
@ -184,7 +187,20 @@ public final class DimensionSelectorUtils
|
|||
return matchNull;
|
||||
} else {
|
||||
for (int i = 0; i < size; ++i) {
|
||||
if (predicateMatchingValueIds.get(row.get(i))) {
|
||||
final int id = row.get(i);
|
||||
final boolean matches;
|
||||
|
||||
if (checkedIds.get(id)) {
|
||||
matches = matchingIds.get(id);
|
||||
} else {
|
||||
matches = predicate.apply(selector.lookupName(id));
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
}
|
||||
}
|
||||
|
||||
if (matches) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -267,16 +267,27 @@ public class SimpleDictionaryEncodedColumn implements DictionaryEncodedColumn<St
|
|||
@Override
|
||||
public ValueMatcher makeValueMatcher(final Predicate<String> predicate)
|
||||
{
|
||||
final BitSet predicateMatchingValueIds = DimensionSelectorUtils.makePredicateMatchingSet(
|
||||
this,
|
||||
predicate
|
||||
);
|
||||
final BitSet checkedIds = new BitSet(getCardinality());
|
||||
final BitSet matchingIds = new BitSet(getCardinality());
|
||||
|
||||
// Lazy matcher; only check an id if matches() is called.
|
||||
return new ValueMatcher()
|
||||
{
|
||||
@Override
|
||||
public boolean matches()
|
||||
{
|
||||
return predicateMatchingValueIds.get(getRowValue());
|
||||
final int id = getRowValue();
|
||||
|
||||
if (checkedIds.get(id)) {
|
||||
return matchingIds.get(id);
|
||||
} else {
|
||||
final boolean matches = predicate.apply(lookupName(id));
|
||||
checkedIds.set(id);
|
||||
if (matches) {
|
||||
matchingIds.set(id);
|
||||
}
|
||||
return matches;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
Loading…
Reference in New Issue