diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 489a4c6e58e..60300d88d0c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -19,6 +19,7 @@ package org.apache.druid.benchmark.query; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -27,6 +28,8 @@ import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.multibindings.MapBinder; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.data.input.impl.DimensionSchema; +import org.apache.druid.data.input.impl.DimensionsSpec; import org.apache.druid.frame.FrameType; import org.apache.druid.frame.testutil.FrameTestUtil; import org.apache.druid.guice.ExpressionModule; @@ -38,6 +41,8 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.math.expr.ExprMacroTable; @@ -49,6 +54,7 @@ import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSket import org.apache.druid.query.aggregation.datasketches.quantiles.sql.DoublesSketchObjectSqlAggregator; import org.apache.druid.query.aggregation.datasketches.theta.sql.ThetaSketchApproxCountDistinctSqlAggregator; import org.apache.druid.query.lookup.LookupExtractor; +import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; @@ -59,6 +65,7 @@ import org.apache.druid.segment.generator.GeneratorBasicSchemas; import org.apache.druid.segment.generator.GeneratorSchemaInfo; import org.apache.druid.segment.generator.SegmentGenerator; import org.apache.druid.segment.join.JoinableFactoryWrapper; +import org.apache.druid.segment.transform.TransformSpec; import org.apache.druid.server.QueryStackTests; import org.apache.druid.server.SpecificSegmentsQuerySegmentWalker; import org.apache.druid.server.security.AuthConfig; @@ -105,6 +112,7 @@ import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * Benchmark that tests various SQL queries. @@ -455,14 +463,65 @@ public class SqlBenchmark private int rowsPerSegment; // Can be "false", "true", or "force" - @Param({"force"}) + @Param({"false", "force"}) private String vectorize; // Can be "none" or "front-coded-N" - @Param({"none", "front-coded-4"}) + @Param({ + "none", + "front-coded-4" + }) private String stringEncoding; - @Param({"28", "29", "30", "31"}) + @Param({ + "explicit", + "auto" + }) + private String schema; + + @Param({ + "0", + "1", + "2", + "3", + "4", + "5", + "6", + "7", + "8", + "9", + "10", + "11", + "12", + "13", + "14", + "15", + "16", + "17", + "18", + "19", + "20", + "21", + "22", + "23", + "24", + "25", + "26", + "27", + "28", + "29", + "30", + "31", + "32", + "33", + "34", + "35", + "36", + "37", + "38", + "39", + "40" + }) private String query; // Can be STORAGE_MMAP, STORAGE_FRAME_ROW, or STORAGE_FRAME_COLUMNAR @@ -483,14 +542,25 @@ public class SqlBenchmark final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); log.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment); - - final QueryableIndex index = segmentGenerator.generate( - dataSegment, - schemaInfo, - IndexSpec.builder().withStringDictionaryEncoding(getStringEncodingStrategy()).build(), - Granularities.NONE, - rowsPerSegment - ); + final QueryableIndex index; + if ("auto".equals(schema)) { + List columnSchemas = schemaInfo.getDimensionsSpec() + .getDimensions() + .stream() + .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .collect(Collectors.toList()); + index = segmentGenerator.generate( + dataSegment, + schemaInfo, + DimensionsSpec.builder().setDimensions(columnSchemas).build(), + TransformSpec.NONE, + IndexSpec.DEFAULT, + Granularities.NONE, + rowsPerSegment + ); + } else { + index = segmentGenerator.generate(dataSegment, schemaInfo, Granularities.NONE, rowsPerSegment); + } final Pair sqlSystem = createSqlSystem( ImmutableMap.of(dataSegment, index), @@ -501,6 +571,33 @@ public class SqlBenchmark plannerFactory = sqlSystem.lhs; engine = sqlSystem.rhs; + + final String sql = QUERIES.get(Integer.parseInt(query)); + final ObjectMapper jsonMapper = CalciteTests.getJsonMapper(); + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, "EXPLAIN PLAN FOR " + sql, ImmutableMap.of("useNativeQueryExplain", true))) { + final PlannerResult plannerResult = planner.plan(); + final Sequence resultSequence = plannerResult.run().getResults(); + final Object[] planResult = resultSequence.toList().get(0); + log.info("Native query plan:\n" + + jsonMapper.writerWithDefaultPrettyPrinter() + .writeValueAsString(jsonMapper.readValue((String) planResult[0], List.class)) + ); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + + try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, ImmutableMap.of())) { + final PlannerResult plannerResult = planner.plan(); + final Sequence resultSequence = plannerResult.run().getResults(); + final Yielder yielder = Yielders.each(resultSequence); + int rowCounter = 0; + while (!yielder.isDone()) { + rowCounter++; + yielder.next(yielder.get()); + } + log.info("Total result row count:" + rowCounter); + } } private StringEncodingStrategy getStringEncodingStrategy() diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java index f21f6de6a69..7b3e71aecec 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlExpressionBenchmark.java @@ -20,7 +20,6 @@ package org.apache.druid.benchmark.query; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.JsonMappingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -212,7 +211,9 @@ public class SqlExpressionBenchmark "SELECT string1, long1 FROM foo WHERE ARRAY_CONTAINS(\"multi-string3\", 100) GROUP BY 1,2", "SELECT string1, long1 FROM foo WHERE ARRAY_OVERLAP(\"multi-string3\", ARRAY[100, 200]) GROUP BY 1,2", // 40: regex filtering - "SELECT string4, COUNT(*) FROM foo WHERE REGEXP_EXTRACT(string1, '^1') IS NOT NULL OR REGEXP_EXTRACT('Z' || string2, '^Z2') IS NOT NULL GROUP BY 1" + "SELECT string4, COUNT(*) FROM foo WHERE REGEXP_EXTRACT(string1, '^1') IS NOT NULL OR REGEXP_EXTRACT('Z' || string2, '^Z2') IS NOT NULL GROUP BY 1", + // 41: complicated filtering + "SELECT string2, SUM(long1) FROM foo WHERE string1 = '1000' AND string5 LIKE '%1%' AND (string3 in ('1', '10', '20', '22', '32') AND long2 IN (1, 19, 21, 23, 25, 26, 46) AND double3 < 1010.0 AND double3 > 1000.0 AND (string4 = '1' OR REGEXP_EXTRACT(string1, '^1') IS NOT NULL OR REGEXP_EXTRACT('Z' || string2, '^Z2') IS NOT NULL)) GROUP BY 1 ORDER BY 2" ); @Param({"5000000"}) @@ -273,7 +274,8 @@ public class SqlExpressionBenchmark "37", "38", "39", - "40" + "40", + "41" }) private String query; @@ -367,9 +369,6 @@ public class SqlExpressionBenchmark .writeValueAsString(jsonMapper.readValue((String) planResult[0], List.class)) ); } - catch (JsonMappingException e) { - throw new RuntimeException(e); - } catch (JsonProcessingException e) { throw new RuntimeException(e); } diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java index 3922ee0e377..1628babb97c 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlNestedDataBenchmark.java @@ -22,6 +22,7 @@ package org.apache.druid.benchmark.query; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import org.apache.druid.common.config.NullHandling; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -76,9 +77,11 @@ import org.openjdk.jmh.annotations.Warmup; import org.openjdk.jmh.infra.Blackhole; import javax.annotation.Nullable; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; @State(Scope.Benchmark) @Fork(value = 1) @@ -193,7 +196,9 @@ public class SqlNestedDataBenchmark "SELECT SUM(JSON_VALUE(nested, '$.long1' RETURNING BIGINT)) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.string5') LIKE '%1%'", // 44, 45 big cardinality like filter + selector filter "SELECT SUM(long1) FROM foo WHERE string5 LIKE '%1%' AND string1 = '1000'", - "SELECT SUM(JSON_VALUE(nested, '$.long1' RETURNING BIGINT)) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.string5') LIKE '%1%' AND JSON_VALUE(nested, '$.nesteder.string1') = '1000'" + "SELECT SUM(JSON_VALUE(nested, '$.long1' RETURNING BIGINT)) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.string5') LIKE '%1%' AND JSON_VALUE(nested, '$.nesteder.string1') = '1000'", + "SELECT SUM(long1) FROM foo WHERE string1 = '1000' AND string5 LIKE '%1%'", + "SELECT SUM(JSON_VALUE(nested, '$.long1' RETURNING BIGINT)) FROM foo WHERE JSON_VALUE(nested, '$.nesteder.string1') = '1000' AND JSON_VALUE(nested, '$.nesteder.string5') LIKE '%1%'" ); @Param({"5000000"}) @@ -212,6 +217,12 @@ public class SqlNestedDataBenchmark }) private String stringEncoding; + @Param({ + "explicit", + "auto" + }) + private String schema; + @Param({ "0", "1", @@ -258,7 +269,9 @@ public class SqlNestedDataBenchmark "42", "43", "44", - "45" + "45", + "46", + "47" }) private String query; @@ -296,11 +309,7 @@ public class SqlNestedDataBenchmark ) ) ); - List dims = ImmutableList.builder() - .addAll(schemaInfo.getDimensionsSpec().getDimensions()) - .add(new AutoTypeColumnSchema("nested", null)) - .build(); - DimensionsSpec dimsSpec = new DimensionsSpec(dims); + StringEncodingStrategy encodingStrategy; @@ -311,15 +320,37 @@ public class SqlNestedDataBenchmark } else { encodingStrategy = new StringEncodingStrategy.Utf8(); } - final QueryableIndex index = segmentGenerator.generate( - dataSegment, - schemaInfo, - dimsSpec, - transformSpec, - IndexSpec.builder().withStringDictionaryEncoding(encodingStrategy).build(), - Granularities.NONE, - rowsPerSegment - ); + final QueryableIndex index; + if ("auto".equals(schema)) { + List columnSchemas = schemaInfo.getDimensionsSpec() + .getDimensions() + .stream() + .map(x -> new AutoTypeColumnSchema(x.getName(), null)) + .collect(Collectors.toList()); + index = segmentGenerator.generate( + dataSegment, + schemaInfo, + DimensionsSpec.builder().setDimensions(columnSchemas).build(), + TransformSpec.NONE, + IndexSpec.builder().withStringDictionaryEncoding(encodingStrategy).build(), + Granularities.NONE, + rowsPerSegment + ); + } else { + Iterable columnSchemas = Iterables.concat( + schemaInfo.getDimensionsSpec().getDimensions(), + Collections.singletonList(new AutoTypeColumnSchema("nested", null)) + ); + index = segmentGenerator.generate( + dataSegment, + schemaInfo, + DimensionsSpec.builder().setDimensions(ImmutableList.copyOf(columnSchemas.iterator())).build(), + TransformSpec.NONE, + IndexSpec.builder().withStringDictionaryEncoding(encodingStrategy).build(), + Granularities.NONE, + rowsPerSegment + ); + } final QueryRunnerFactoryConglomerate conglomerate = QueryStackTests.createQueryRunnerFactoryConglomerate( closer, diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/BitmapFactory.java b/processing/src/main/java/org/apache/druid/collections/bitmap/BitmapFactory.java index c0663757530..706177928c0 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/BitmapFactory.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/BitmapFactory.java @@ -65,7 +65,7 @@ public interface BitmapFactory /** * Compute the intersection (bitwise-AND) of a set of bitmaps. They are assumed to be - * instances of of the proper WrappedConciseBitmap otherwise a ClassCastException + * instances of the proper WrappedConciseBitmap otherwise a ClassCastException * is thrown. * * @param b input ImmutableGenericBitmap objects diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java index 5dbdad93de5..89a124c0f51 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/ImmutableBitmap.java @@ -80,4 +80,12 @@ public interface ImmutableBitmap */ ImmutableBitmap intersection(ImmutableBitmap otherBitmap); + /** + * Compute the bitwise-or of this bitmap with another bitmap. A new bitmap is generated. + * + * Note that the other bitmap should be of the same class instance. + * + * @param otherBitmap other bitmap + */ + ImmutableBitmap union(ImmutableBitmap otherBitmap); } diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedConciseBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedConciseBitmap.java index 1f881681082..142d54b620e 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedConciseBitmap.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedConciseBitmap.java @@ -132,6 +132,15 @@ public class WrappedConciseBitmap implements MutableBitmap return new WrappedConciseBitmap(bitmap.clone().intersection(unwrappedOtherBitmap)); } + @Override + public ImmutableBitmap union(ImmutableBitmap otherBitmap) + { + WrappedConciseBitmap other = (WrappedConciseBitmap) otherBitmap; + ConciseSet clone = bitmap.clone(); + clone.addAll(other.bitmap); + return new WrappedConciseBitmap(clone); + } + @Override public boolean get(int value) { diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableBitSetBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableBitSetBitmap.java index 4b8da4f6285..b52fc0bf2c1 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableBitSetBitmap.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableBitSetBitmap.java @@ -80,6 +80,7 @@ public class WrappedImmutableBitSetBitmap implements ImmutableBitmap return bitmap.isEmpty(); } + @Override public ImmutableBitmap union(ImmutableBitmap otherBitmap) { WrappedBitSetBitmap retval = new WrappedBitSetBitmap((BitSet) bitmap.clone()); diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableConciseBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableConciseBitmap.java index c0cf3435ab8..aa9f1e38e59 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableConciseBitmap.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableConciseBitmap.java @@ -98,8 +98,13 @@ public class WrappedImmutableConciseBitmap implements ImmutableBitmap public ImmutableBitmap intersection(ImmutableBitmap otherBitmap) { WrappedImmutableConciseBitmap other = (WrappedImmutableConciseBitmap) otherBitmap; - ImmutableConciseSet unwrappedOtherBitmap = other.bitmap; - return new WrappedImmutableConciseBitmap(ImmutableConciseSet.intersection(bitmap, unwrappedOtherBitmap)); + return new WrappedImmutableConciseBitmap(ImmutableConciseSet.intersection(bitmap, other.bitmap)); } + @Override + public ImmutableBitmap union(ImmutableBitmap otherBitmap) + { + WrappedImmutableConciseBitmap other = (WrappedImmutableConciseBitmap) otherBitmap; + return new WrappedImmutableConciseBitmap(ImmutableConciseSet.union(bitmap, other.bitmap)); + } } diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java index 023d2fbe2ef..97f7b65cfe2 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedImmutableRoaringBitmap.java @@ -112,8 +112,13 @@ public class WrappedImmutableRoaringBitmap implements ImmutableBitmap public ImmutableBitmap intersection(ImmutableBitmap otherBitmap) { WrappedImmutableRoaringBitmap other = (WrappedImmutableRoaringBitmap) otherBitmap; - ImmutableRoaringBitmap unwrappedOtherBitmap = other.bitmap; - return new WrappedImmutableRoaringBitmap(ImmutableRoaringBitmap.and(bitmap, unwrappedOtherBitmap)); + return new WrappedImmutableRoaringBitmap(ImmutableRoaringBitmap.and(bitmap, other.bitmap)); } + @Override + public ImmutableBitmap union(ImmutableBitmap otherBitmap) + { + WrappedImmutableRoaringBitmap other = (WrappedImmutableRoaringBitmap) otherBitmap; + return new WrappedImmutableRoaringBitmap(ImmutableRoaringBitmap.or(bitmap, other.bitmap)); + } } diff --git a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedRoaringBitmap.java b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedRoaringBitmap.java index faeaac5cc02..bc75d852b15 100644 --- a/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedRoaringBitmap.java +++ b/processing/src/main/java/org/apache/druid/collections/bitmap/WrappedRoaringBitmap.java @@ -256,6 +256,16 @@ public class WrappedRoaringBitmap implements MutableBitmap return new WrappedImmutableRoaringBitmap(MutableRoaringBitmap.and(writer.get(), unwrappedOtherBitmap)); } + @Override + public ImmutableBitmap union(ImmutableBitmap otherBitmap) + { + initializeWriterIfNeeded(); + WrappedRoaringBitmap other = (WrappedRoaringBitmap) otherBitmap; + other.initializeWriterIfNeeded(); + MutableRoaringBitmap unwrappedOtherBitmap = other.writer.get(); + return new WrappedImmutableRoaringBitmap(MutableRoaringBitmap.or(writer.get(), unwrappedOtherBitmap)); + } + @Override public boolean get(int value) { diff --git a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java index 8c2f9e9b9e9..3497ed39898 100644 --- a/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java +++ b/processing/src/main/java/org/apache/druid/frame/segment/columnar/FrameCursorFactory.java @@ -34,6 +34,7 @@ import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.segment.ColumnCache; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.CursorFactory; @@ -149,10 +150,10 @@ public class FrameCursorFactory implements CursorFactory if (filterToUse == null) { return new FrameVectorCursor(baseOffset, baseColumnSelectorFactory, closer); } else { + final VectorValueMatcher matcher = filterToUse.makeVectorMatcher(baseColumnSelectorFactory); final FilteredVectorOffset filteredOffset = FilteredVectorOffset.create( baseOffset, - baseColumnSelectorFactory, - filterToUse + matcher ); final VectorColumnSelectorFactory filteredColumnSelectorFactory = new QueryableIndexVectorColumnSelectorFactory( diff --git a/processing/src/main/java/org/apache/druid/math/expr/ExpressionPredicateIndexSupplier.java b/processing/src/main/java/org/apache/druid/math/expr/ExpressionPredicateIndexSupplier.java index 6638af3a124..0afe36229e0 100644 --- a/processing/src/main/java/org/apache/druid/math/expr/ExpressionPredicateIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/math/expr/ExpressionPredicateIndexSupplier.java @@ -28,7 +28,7 @@ import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.index.BitmapColumnIndex; -import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex; +import org.apache.druid.segment.index.DictionaryScanningBitmapIndex; import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; @@ -78,7 +78,7 @@ public class ExpressionPredicateIndexSupplier implements ColumnIndexSupplier final java.util.function.Function> evalFunction = inputValue -> expr.eval(InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue)); - return new SimpleImmutableBitmapIterableIndex() + return new DictionaryScanningBitmapIndex(inputColumnIndexes.getCardinality()) { @Override public Iterable getBitmapIterable(boolean includeUnknown) diff --git a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java index 11eb0ccdd33..db734651797 100644 --- a/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/DefaultQueryMetrics.java @@ -25,11 +25,9 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; -import org.apache.druid.query.filter.Filter; import org.joda.time.Interval; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; @@ -192,18 +190,6 @@ public class DefaultQueryMetrics> implements QueryMet setDimension("segment", segmentIdentifier); } - @Override - public void preFilters(List preFilters) - { - // Emit nothing by default. - } - - @Override - public void postFilters(List postFilters) - { - // Emit nothing by default. - } - @Override public void identity(String identity) { diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java index ca645d16e97..de5859e8820 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingConfig.java @@ -177,12 +177,6 @@ public class DruidProcessingConfig implements ColumnConfig return indexes.getSkipValueRangeIndexScale(); } - @Override - public double skipValuePredicateIndexScale() - { - return indexes.getSkipValuePredicateIndexScale(); - } - public boolean isNumThreadsConfigured() { return numThreadsConfigured; diff --git a/processing/src/main/java/org/apache/druid/query/DruidProcessingIndexesConfig.java b/processing/src/main/java/org/apache/druid/query/DruidProcessingIndexesConfig.java index 6afa20418e9..54ba6b6eb10 100644 --- a/processing/src/main/java/org/apache/druid/query/DruidProcessingIndexesConfig.java +++ b/processing/src/main/java/org/apache/druid/query/DruidProcessingIndexesConfig.java @@ -31,37 +31,24 @@ public class DruidProcessingIndexesConfig @JsonProperty private final double skipValueRangeIndexScale; - @JsonProperty - private final double skipValuePredicateIndexScale; - @JsonCreator public DruidProcessingIndexesConfig( - @JsonProperty("skipValueRangeIndexScale") @Nullable Double skipValueRangeIndexScale, - @JsonProperty("skipValuePredicateIndexScale") @Nullable Double skipValuePredicateIndexScale + @JsonProperty("skipValueRangeIndexScale") @Nullable Double skipValueRangeIndexScale ) { this.skipValueRangeIndexScale = Configs.valueOrDefault( skipValueRangeIndexScale, ColumnConfig.DEFAULT_SKIP_VALUE_RANGE_INDEX_SCALE ); - this.skipValuePredicateIndexScale = Configs.valueOrDefault( - skipValuePredicateIndexScale, - ColumnConfig.DEFAULT_SKIP_VALUE_PREDICATE_INDEX_SCALE - ); } public DruidProcessingIndexesConfig() { - this(null, null); + this(null); } public double getSkipValueRangeIndexScale() { return skipValueRangeIndexScale; } - - public double getSkipValuePredicateIndexScale() - { - return skipValuePredicateIndexScale; - } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java index ddbcfc0ed53..6dfad11fae4 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/QueryMetrics.java @@ -24,6 +24,7 @@ import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.FilterBundle; import org.apache.druid.query.search.SearchQueryMetricsFactory; import java.util.List; @@ -242,9 +243,36 @@ public interface QueryMetrics> void segment(String segmentIdentifier); - void preFilters(List preFilters); + /** + * @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were + * used to construct {@link org.apache.druid.segment.BitmapOffset} or + * {@link org.apache.druid.segment.vector.BitmapVectorOffset}. + * This method will be removed in a future Druid release + */ + @Deprecated + @SuppressWarnings({"unreachable", "unused"}) + default void preFilters(List preFilters) + { + // do nothing, nothing calls this + } - void postFilters(List postFilters); + /** + * @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were + * used as value matchers for {@link org.apache.druid.segment.FilteredOffset} or + * {@link org.apache.druid.segment.vector.FilteredVectorOffset} + * This method will be removed in a future Druid release + */ + @Deprecated + @SuppressWarnings({"unreachable", "unused"}) + default void postFilters(List postFilters) + { + // do nothing, nothing calls this + } + + default void filterBundle(FilterBundle.BundleInfo bundleInfo) + { + // Emit nothing by default. + } /** * Sets identity of the requester for a query. See {@code AuthenticationResult}. @@ -265,10 +293,10 @@ public interface QueryMetrics> void parallelMergeParallelism(int parallelism); /** - * Creates a {@link BitmapResultFactory} which may record some information along bitmap construction from {@link - * #preFilters(List)}. The returned BitmapResultFactory may add some dimensions to this QueryMetrics from it's {@link - * BitmapResultFactory#toImmutableBitmap(Object)} method. See {@link BitmapResultFactory} Javadoc for more - * information. + * Creates a {@link BitmapResultFactory} which may record some information along bitmap construction from + * {@link #filterBundle(FilterBundle.BundleInfo)}. The returned BitmapResultFactory may add some dimensions to this + * QueryMetrics from it's {@link BitmapResultFactory#toImmutableBitmap(Object)} method. See + * {@link BitmapResultFactory} Javadoc for more information. */ BitmapResultFactory makeBitmapResultFactory(BitmapFactory factory); @@ -348,8 +376,8 @@ public interface QueryMetrics> QueryMetrics reportNodeBytes(long byteCount); /** - * Reports the time spent constructing bitmap from {@link #preFilters(List)} of the query. Not reported, if there are - * no preFilters. + * Reports the time spent constructing bitmap from {@link #filterBundle(FilterBundle.BundleInfo)} of the query. Not + * reported, if there are no indexes. */ QueryMetrics reportBitmapConstructionTime(long timeNs); @@ -359,8 +387,8 @@ public interface QueryMetrics> QueryMetrics reportSegmentRows(long numRows); /** - * Reports the number of rows to scan in the segment after applying {@link #preFilters(List)}. If the are no - * preFilters, this metric is equal to {@link #reportSegmentRows(long)}. + * Reports the number of rows to scan in the segment after applying {@link #filterBundle(FilterBundle.BundleInfo)}. + * If the are no indexes, this metric is equal to {@link #reportSegmentRows(long)}. */ QueryMetrics reportPreFilteredRows(long numRows); diff --git a/processing/src/main/java/org/apache/druid/query/filter/BooleanFilter.java b/processing/src/main/java/org/apache/druid/query/filter/BooleanFilter.java index 0ff864258a9..e14e8b8d5e3 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/BooleanFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/BooleanFilter.java @@ -19,16 +19,12 @@ package org.apache.druid.query.filter; -import org.apache.druid.segment.ColumnSelectorFactory; - import java.util.HashSet; import java.util.LinkedHashSet; import java.util.Set; public interface BooleanFilter extends Filter { - ValueMatcher[] EMPTY_VALUE_MATCHER_ARRAY = new ValueMatcher[0]; - /** * Returns the child filters for this filter. * @@ -37,28 +33,6 @@ public interface BooleanFilter extends Filter */ LinkedHashSet 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 ColumnIndexSelector, and - * return a ValueMatcher that checks the current row offset, created using the bitmap index. - * - * @param selector Object used to retrieve bitmap indexes - * @param columnSelectorFactory Object used to select columns for making ValueMatchers - * @param rowOffsetMatcherFactory Object used to create RowOffsetMatchers - * - * @return ValueMatcher that applies this filter - */ - ValueMatcher makeMatcher( - ColumnIndexSelector selector, - ColumnSelectorFactory columnSelectorFactory, - RowOffsetMatcherFactory rowOffsetMatcherFactory - ); - @Override default Set getRequiredColumns() { @@ -68,5 +42,4 @@ public interface BooleanFilter extends Filter } return allColumns; } - } diff --git a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java index fbd940c0f30..e78bc987029 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/DimFilter.java @@ -126,7 +126,7 @@ public interface DimFilter extends Cacheable /** * Append dimension name OR {@link ExtractionFn#toString()} with dimension wrapped in parenthesis */ - DimFilterToStringBuilder appendDimension(String dimension, @Nullable ExtractionFn extractionFn) + public DimFilterToStringBuilder appendDimension(String dimension, @Nullable ExtractionFn extractionFn) { if (extractionFn != null) { builder.append(extractionFn).append("("); @@ -143,7 +143,7 @@ public interface DimFilter extends Cacheable /** * Add "=" expression */ - DimFilterToStringBuilder appendEquals(String value) + public DimFilterToStringBuilder appendEquals(String value) { builder.append(" = ").append(value); return this; @@ -152,7 +152,7 @@ public interface DimFilter extends Cacheable /** * Add filter tuning to {@link #builder} if tuning exists */ - DimFilterToStringBuilder appendFilterTuning(@Nullable FilterTuning tuning) + public DimFilterToStringBuilder appendFilterTuning(@Nullable FilterTuning tuning) { if (tuning != null) { builder.append(" (filterTuning=").append(tuning).append(")"); @@ -164,7 +164,7 @@ public interface DimFilter extends Cacheable /** * Generic passthrough to {@link StringBuilder#append} */ - DimFilterToStringBuilder append(T s) + public DimFilterToStringBuilder append(T s) { builder.append(s); return this; diff --git a/processing/src/main/java/org/apache/druid/query/filter/Filter.java b/processing/src/main/java/org/apache/druid/query/filter/Filter.java index e7ef20790c0..4a83b4c1b8a 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/Filter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/Filter.java @@ -20,6 +20,8 @@ package org.apache.druid.query.filter; import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.UOE; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.filter.vector.VectorValueMatcher; @@ -35,6 +37,79 @@ import java.util.Set; @SubclassesMustOverrideEqualsAndHashCode public interface Filter { + /** + * Compute indexes and build a container {@link FilterBundle} to be used during + * {@link org.apache.druid.segment.Cursor} or {@link org.apache.druid.segment.vector.VectorCursor} creation, combining + * the computed outputs of {@link #getBitmapColumnIndex(ColumnIndexSelector)} as well as references to + * {@link #makeMatcher(ColumnSelectorFactory)} and {@link #makeVectorMatcher(VectorColumnSelectorFactory)}. + *

+ * Filters populating the {@link FilterBundle} container should only set the values which MUST be evaluated by the + * cursor. If both are set, the cursor will effectively perform a logical AND to combine them. + * See {@link FilterBundle} for additional details. + * + * @param columnIndexSelector - provides {@link org.apache.druid.segment.column.ColumnIndexSupplier} to fetch column + * indexes and {@link org.apache.druid.collections.bitmap.BitmapFactory} to manipulate + * them + * @param bitmapResultFactory - wrapper for {@link ImmutableBitmap} operations to tie into + * {@link org.apache.druid.query.QueryMetrics} and build the output indexes + * @param selectionRowCount - number of rows selected so far by any previous bundle computations + * @param totalRowCount - total number of rows to be scanned if no indexes are applied + * @param includeUnknown - mapping for Druid native two state logic system into SQL three-state logic system. If + * set to true, bitmaps returned by this method should include true bits for any rows + * where the matching result is 'unknown', such as from the input being null valued. + * See {@link NullHandling#useThreeValueLogic()} + * @return - {@link FilterBundle} containing any indexes and/or matchers that are needed to build + * a cursor + * @param - Type of {@link BitmapResultFactory} results, {@link ImmutableBitmap} by default + */ + default FilterBundle makeFilterBundle( + ColumnIndexSelector columnIndexSelector, + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + final FilterBundle.IndexBundle indexBundle; + final boolean needMatcher; + final BitmapColumnIndex columnIndex = getBitmapColumnIndex(columnIndexSelector); + if (columnIndex != null) { + final long bitmapConstructionStartNs = System.nanoTime(); + final T result = columnIndex.computeBitmapResult( + bitmapResultFactory, + selectionRowCount, + totalRowCount, + includeUnknown + ); + final long totalConstructionTimeNs = System.nanoTime() - bitmapConstructionStartNs; + if (result == null) { + indexBundle = null; + } else { + final ImmutableBitmap bitmap = bitmapResultFactory.toImmutableBitmap(result); + indexBundle = new FilterBundle.SimpleIndexBundle( + new FilterBundle.IndexBundleInfo(this::toString, bitmap.size(), totalConstructionTimeNs, null), + bitmap, + columnIndex.getIndexCapabilities() + ); + } + needMatcher = result == null || !columnIndex.getIndexCapabilities().isExact(); + } else { + indexBundle = null; + needMatcher = true; + } + final FilterBundle.SimpleMatcherBundle matcherBundle; + if (needMatcher) { + matcherBundle = new FilterBundle.SimpleMatcherBundle( + new FilterBundle.MatcherBundleInfo(this::toString, null, null), + this::makeMatcher, + this::makeVectorMatcher + ); + } else { + matcherBundle = null; + } + return new FilterBundle(indexBundle, matcherBundle); + } + /** * Returns a {@link BitmapColumnIndex} if this filter supports using a bitmap index for filtering for the given input * {@link ColumnIndexSelector}. The {@link BitmapColumnIndex} can be used to compute into a bitmap indicating rows diff --git a/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java b/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java new file mode 100644 index 00000000000..78d666c61cb --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/filter/FilterBundle.java @@ -0,0 +1,358 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.query.filter; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; +import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.ColumnIndexCapabilities; +import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; +import org.apache.druid.segment.data.Offset; +import org.apache.druid.segment.filter.FalseFilter; +import org.apache.druid.segment.vector.ReadableVectorOffset; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; + +/** + * FilterBundle is a container for all the goodies used for producing filtered cursors, a {@link ImmutableBitmap} if + * the filter can use an index, and/or a {@link MatcherBundle} which contains functions to build {@link ValueMatcher} + * and {@link VectorValueMatcher} for any filters which must be evaluated row by row during the cursor scan. Cursors + * will use everything that is non-null, and at least one of index or matcher bundle MUST be set. If both index and + * matcher is present, the cursor will effectively perform a logical AND operation; i.e. if the index matches a row + * then the value matcher must also match the row for the cursor to provide it to the selectors built on top of it. + *

+ * There are a few cases where the filter should set both indexes and matchers. For example, if the filter is a + * composite filter which can be partitioned, such as {@link org.apache.druid.segment.filter.AndFilter}, then the filter + * can be partitioned due to the intersection nature of AND, so the index can be set to reduce the number of rows and + * the matcher bundle will build a matcher which will filter the remainder. This strategy of having both an index and a + * matcher can also can model the case where the index is an inexact match and so a matcher must be used to ensure that + * the remaining values actually match the filter. + */ +public class FilterBundle +{ + public static FilterBundle allFalse(long constructionTime, ImmutableBitmap emptyBitmap) + { + return new FilterBundle( + new FilterBundle.SimpleIndexBundle( + new FilterBundle.IndexBundleInfo(() -> FalseFilter.instance().toString(), 0, constructionTime, null), + emptyBitmap, + SimpleColumnIndexCapabilities.getConstant() + ), + null + ); + } + + @Nullable + private final IndexBundle indexBundle; + @Nullable + private final MatcherBundle matcherBundle; + + public FilterBundle( + @Nullable IndexBundle index, + @Nullable MatcherBundle matcherBundle + ) + { + Preconditions.checkArgument( + index != null || matcherBundle != null, + "At least one of index or matcher must be not null" + ); + this.indexBundle = index; + this.matcherBundle = matcherBundle; + } + + + @Nullable + public IndexBundle getIndex() + { + return indexBundle; + } + + @Nullable + public MatcherBundle getMatcherBundle() + { + return matcherBundle; + } + + public BundleInfo getInfo() + { + return new BundleInfo( + indexBundle == null ? null : indexBundle.getIndexInfo(), + matcherBundle == null ? null : matcherBundle.getMatcherInfo() + ); + } + + public boolean hasIndex() + { + return indexBundle != null; + } + + public boolean hasMatcher() + { + return matcherBundle != null; + } + + public interface IndexBundle + { + IndexBundleInfo getIndexInfo(); + ImmutableBitmap getBitmap(); + ColumnIndexCapabilities getIndexCapabilities(); + } + + /** + * Builder of {@link ValueMatcher} and {@link VectorValueMatcher}. The + * {@link #valueMatcher(ColumnSelectorFactory, Offset, boolean)} function also passes in the base offset and whether + * the offset is 'descending' or not, to allow filters more flexibility in value matcher creation. + * {@link org.apache.druid.segment.filter.OrFilter} uses these extra parameters to allow partial use of indexes to + * create a synthetic value matcher that checks if the row is set in the bitmap, instead of purely using value + * matchers, with {@link org.apache.druid.segment.filter.OrFilter#convertIndexToValueMatcher}. + */ + public interface MatcherBundle + { + MatcherBundleInfo getMatcherInfo(); + ValueMatcher valueMatcher(ColumnSelectorFactory selectorFactory, Offset baseOffset, boolean descending); + VectorValueMatcher vectorMatcher(VectorColumnSelectorFactory selectorFactory, ReadableVectorOffset baseOffset); + } + + public static class SimpleIndexBundle implements IndexBundle + { + private final IndexBundleInfo info; + private final ImmutableBitmap index; + private final ColumnIndexCapabilities indexCapabilities; + + public SimpleIndexBundle(IndexBundleInfo info, ImmutableBitmap index, ColumnIndexCapabilities indexCapabilities) + { + this.info = Preconditions.checkNotNull(info); + this.index = Preconditions.checkNotNull(index); + this.indexCapabilities = Preconditions.checkNotNull(indexCapabilities); + } + + @Override + public IndexBundleInfo getIndexInfo() + { + return info; + } + + @Override + public ImmutableBitmap getBitmap() + { + return index; + } + + @Override + public ColumnIndexCapabilities getIndexCapabilities() + { + return indexCapabilities; + } + } + + public static class SimpleMatcherBundle implements MatcherBundle + { + private final MatcherBundleInfo matcherInfo; + private final Function matcherFn; + private final Function vectorMatcherFn; + + public SimpleMatcherBundle( + MatcherBundleInfo matcherInfo, + Function matcherFn, + Function vectorMatcherFn + ) + { + this.matcherInfo = Preconditions.checkNotNull(matcherInfo); + this.matcherFn = Preconditions.checkNotNull(matcherFn); + this.vectorMatcherFn = Preconditions.checkNotNull(vectorMatcherFn); + } + + @Override + public MatcherBundleInfo getMatcherInfo() + { + return matcherInfo; + } + + @Override + public ValueMatcher valueMatcher( + ColumnSelectorFactory selectorFactory, + Offset baseOffset, + boolean descending + ) + { + return matcherFn.apply(selectorFactory); + } + + @Override + public VectorValueMatcher vectorMatcher(VectorColumnSelectorFactory selectorFactory, ReadableVectorOffset baseOffset) + { + return vectorMatcherFn.apply(selectorFactory); + } + } + + public static class BundleInfo + { + private final IndexBundleInfo index; + private final MatcherBundleInfo matcher; + + @JsonCreator + public BundleInfo( + @JsonProperty("index") @Nullable IndexBundleInfo index, + @JsonProperty("matcher") @Nullable MatcherBundleInfo matcher + ) + { + this.index = index; + this.matcher = matcher; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public IndexBundleInfo getIndex() + { + return index; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public MatcherBundleInfo getMatcher() + { + return matcher; + } + + @Override + public String toString() + { + return "{index=" + index + ", matcher=" + matcher + '}'; + } + } + + public static class IndexBundleInfo + { + private final Supplier filter; + private final List indexes; + private final int selectionSize; + private final long buildTimeNs; + + public IndexBundleInfo( + Supplier filterString, + int selectionSize, + long buildTimeNs, + @Nullable List indexes + ) + { + this.filter = filterString; + this.selectionSize = selectionSize; + this.buildTimeNs = buildTimeNs; + this.indexes = indexes; + } + + @JsonProperty + public String getFilter() + { + return filter.get(); + } + + @JsonProperty + public int getSelectionSize() + { + return selectionSize; + } + + @JsonProperty + public long getBuildTimeNs() + { + return buildTimeNs; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getIndexes() + { + return indexes; + } + + @Override + public String toString() + { + return "{" + + "filter=\"" + filter.get() + '\"' + + ", selectionSize=" + selectionSize + + ", buildTime=" + TimeUnit.NANOSECONDS.toMicros(buildTimeNs) + "μs" + + (indexes != null ? ", indexes=" + indexes : "") + + '}'; + } + } + + public static class MatcherBundleInfo + { + private final Supplier filter; + @Nullable + final List matchers; + + @Nullable + private final IndexBundleInfo partialIndex; + + public MatcherBundleInfo( + Supplier filter, + @Nullable IndexBundleInfo partialIndex, + @Nullable List matchers + ) + { + this.filter = filter; + this.matchers = matchers; + this.partialIndex = partialIndex; + } + + @JsonProperty + public String getFilter() + { + return filter.get(); + } + + @Nullable + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public IndexBundleInfo getPartialIndex() + { + return partialIndex; + } + + @JsonProperty + @JsonInclude(JsonInclude.Include.NON_NULL) + public List getMatchers() + { + return matchers; + } + + @Override + public String toString() + { + return "{" + + "filter=\"" + filter.get() + '\"' + + (partialIndex != null ? ", partialIndex=" + partialIndex : "") + + (matchers != null ? ", matchers=" + matchers : "") + + '}'; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java b/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java index 415d2e8369b..831d50261e2 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java +++ b/processing/src/main/java/org/apache/druid/query/filter/FilterTuning.java @@ -23,16 +23,17 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.annotations.SubclassesMustOverrideEqualsAndHashCode; +import org.apache.druid.query.BitmapResultFactory; import javax.annotation.Nullable; import java.util.Objects; /** * This class provides a mechanism to influence whether or not indexes are used for a {@link Filter} during processing - * by {@link org.apache.druid.segment.FilterAnalysis#analyzeFilter} (i.e. will a {@link Filter} be a "pre" - * filter in which we union indexes for all values that match the filter to create a - * {@link org.apache.druid.segment.BitmapOffset}/{@link org.apache.druid.segment.vector.BitmapVectorOffset}, or will it - * be used as a "post" filter and evaluated while scanning row values from the + * by {@link Filter#makeFilterBundle(ColumnIndexSelector, BitmapResultFactory, int, int, boolean)} + * (i.e. will a {@link Filter} be a "pre" filter in which we union indexes for all values that match the filter to + * create a {@link org.apache.druid.segment.BitmapOffset}/{@link org.apache.druid.segment.vector.BitmapVectorOffset}, + * or will it be used as a "post" filter and evaluated while scanning row values from the * {@link org.apache.druid.segment.FilteredOffset}/{@link org.apache.druid.segment.vector.FilteredVectorOffset}. * * This is currently only manually supplied by the user by adding to a {@link DimFilter} which will pass through to the diff --git a/processing/src/main/java/org/apache/druid/query/filter/LikeDimFilter.java b/processing/src/main/java/org/apache/druid/query/filter/LikeDimFilter.java index 3334878177b..c9a1ec7880f 100644 --- a/processing/src/main/java/org/apache/druid/query/filter/LikeDimFilter.java +++ b/processing/src/main/java/org/apache/druid/query/filter/LikeDimFilter.java @@ -217,12 +217,16 @@ public class LikeDimFilter extends AbstractOptimizableDimFilter implements DimFi // Regex pattern that describes matching strings. private final Pattern pattern; + private final String likePattern; + private LikeMatcher( + final String likePattern, final SuffixMatch suffixMatch, final String prefix, final Pattern pattern ) { + this.likePattern = likePattern; this.suffixMatch = Preconditions.checkNotNull(suffixMatch, "suffixMatch"); this.prefix = NullHandling.nullToEmptyIfNeeded(prefix); this.pattern = Preconditions.checkNotNull(pattern, "pattern"); @@ -263,7 +267,7 @@ public class LikeDimFilter extends AbstractOptimizableDimFilter implements DimFi } } - return new LikeMatcher(suffixMatch, prefix.toString(), Pattern.compile(regex.toString(), Pattern.DOTALL)); + return new LikeMatcher(likePattern, suffixMatch, prefix.toString(), Pattern.compile(regex.toString(), Pattern.DOTALL)); } private static void addPatternCharacter(final StringBuilder patternBuilder, final char c) @@ -413,5 +417,11 @@ public class LikeDimFilter extends AbstractOptimizableDimFilter implements DimFi { return Objects.hash(getSuffixMatch(), getPrefix(), pattern.toString()); } + + @Override + public String toString() + { + return likePattern; + } } } diff --git a/processing/src/main/java/org/apache/druid/query/filter/RowOffsetMatcherFactory.java b/processing/src/main/java/org/apache/druid/query/filter/RowOffsetMatcherFactory.java deleted file mode 100644 index 1e2f5a4f26b..00000000000 --- a/processing/src/main/java/org/apache/druid/query/filter/RowOffsetMatcherFactory.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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 org.apache.druid.query.filter; - -import org.apache.druid.collections.bitmap.ImmutableBitmap; - -/** - */ -public interface RowOffsetMatcherFactory -{ - ValueMatcher makeRowOffsetMatcher(ImmutableBitmap bitmap); -} diff --git a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java index eeb9b976a99..ef484df5f18 100644 --- a/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/search/DefaultSearchQueryMetrics.java @@ -25,9 +25,7 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.query.Query; import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; - -import java.util.List; +import org.apache.druid.query.filter.FilterBundle; /** * This class is implemented with delegation to another QueryMetrics for compatibility, see "Making subinterfaces of @@ -154,15 +152,9 @@ public class DefaultSearchQueryMetrics implements SearchQueryMetrics } @Override - public void preFilters(List preFilters) + public void filterBundle(FilterBundle.BundleInfo bundleInfo) { - delegateQueryMetrics.preFilters(preFilters); - } - - @Override - public void postFilters(List postFilters) - { - delegateQueryMetrics.postFilters(postFilters); + delegateQueryMetrics.filterBundle(bundleInfo); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryMetrics.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryMetrics.java index 7e3e8ca3a9a..bf458ce7115 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryMetrics.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryMetrics.java @@ -22,6 +22,7 @@ package org.apache.druid.query.topn; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.guice.annotations.PublicApi; import org.apache.druid.query.QueryMetrics; +import org.apache.druid.query.filter.FilterBundle; import org.apache.druid.segment.ColumnValueSelector; import org.apache.druid.segment.Cursor; @@ -90,8 +91,8 @@ public interface TopNQueryMetrics extends QueryMetrics /** * Called with the number of rows, processed via each cursor, processed for the query within the segment. The total * number of processed rows, reported via this method for a TopNQueryMetrics instance, is smaller or equal to - * {@link #reportPreFilteredRows(long)}, because {@link #postFilters} are additionally applied. If there - * are no postFilters, preFilteredRows and processedRows are equal. + * {@link #reportPreFilteredRows(long)}, because matchers from {@link #filterBundle(FilterBundle.BundleInfo)} are + * additionally applied. If there are no postFilters, preFilteredRows and processedRows are equal. */ @PublicApi TopNQueryMetrics addProcessedRows(long numRows); diff --git a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java index 33cb490b912..ab552f0d5a3 100644 --- a/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/BitmapOffset.java @@ -127,7 +127,7 @@ public class BitmapOffset extends Offset private final int valueForReset; private int value; - static IntIterator getReverseBitmapOffsetIterator(ImmutableBitmap bitmapIndex) + public static IntIterator getReverseBitmapOffsetIterator(ImmutableBitmap bitmapIndex) { ImmutableBitmap roaringBitmap = bitmapIndex; if (!(bitmapIndex instanceof WrappedImmutableRoaringBitmap)) { diff --git a/processing/src/main/java/org/apache/druid/segment/FilterAnalysis.java b/processing/src/main/java/org/apache/druid/segment/FilterAnalysis.java deleted file mode 100644 index c700c5b5e86..00000000000 --- a/processing/src/main/java/org/apache/druid/segment/FilterAnalysis.java +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF 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 org.apache.druid.segment; - -import org.apache.druid.collections.bitmap.ImmutableBitmap; -import org.apache.druid.query.BitmapResultFactory; -import org.apache.druid.query.DefaultBitmapResultFactory; -import org.apache.druid.query.QueryMetrics; -import org.apache.druid.query.filter.Filter; -import org.apache.druid.segment.filter.AndFilter; -import org.apache.druid.segment.filter.Filters; -import org.apache.druid.segment.index.BitmapColumnIndex; - -import javax.annotation.Nullable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - -public class FilterAnalysis -{ - @SuppressWarnings("rawtypes") - public static FilterAnalysis analyzeFilter( - @Nullable final Filter filter, - ColumnSelectorColumnIndexSelector indexSelector, - @Nullable QueryMetrics queryMetrics, - int numRows - ) - { - - /* - * 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 non-null value for getBitmapColumnIndex) - * 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 List preFilters; - final List postFilters = new ArrayList<>(); - int preFilteredRows = numRows; - if (filter == null) { - preFilters = Collections.emptyList(); - } else { - preFilters = new ArrayList<>(); - - if (filter instanceof AndFilter) { - // If we get an AndFilter, we can split the subfilters across both filtering stages - for (Filter subfilter : ((AndFilter) filter).getFilters()) { - - final BitmapColumnIndex columnIndex = subfilter.getBitmapColumnIndex(indexSelector); - - if (columnIndex == null) { - postFilters.add(subfilter); - } else { - preFilters.add(subfilter); - if (!columnIndex.getIndexCapabilities().isExact()) { - postFilters.add(subfilter); - } - } - } - } else { - // If we get an OrFilter or a single filter, handle the filter in one stage - final BitmapColumnIndex columnIndex = filter.getBitmapColumnIndex(indexSelector); - if (columnIndex == null) { - postFilters.add(filter); - } else { - preFilters.add(filter); - if (!columnIndex.getIndexCapabilities().isExact()) { - postFilters.add(filter); - } - } - } - } - - final ImmutableBitmap preFilterBitmap; - if (preFilters.isEmpty()) { - preFilterBitmap = null; - } else { - if (queryMetrics != null) { - BitmapResultFactory bitmapResultFactory = - queryMetrics.makeBitmapResultFactory(indexSelector.getBitmapFactory()); - long bitmapConstructionStartNs = System.nanoTime(); - // Use AndFilter.getBitmapIndex to intersect the preFilters to get its short-circuiting behavior. - preFilterBitmap = AndFilter.getBitmapIndex(indexSelector, bitmapResultFactory, preFilters); - preFilteredRows = preFilterBitmap.size(); - queryMetrics.reportBitmapConstructionTime(System.nanoTime() - bitmapConstructionStartNs); - } else { - BitmapResultFactory bitmapResultFactory = new DefaultBitmapResultFactory(indexSelector.getBitmapFactory()); - preFilterBitmap = AndFilter.getBitmapIndex(indexSelector, bitmapResultFactory, preFilters); - } - } - - if (queryMetrics != null) { - queryMetrics.preFilters(new ArrayList<>(preFilters)); - queryMetrics.postFilters(postFilters); - queryMetrics.reportSegmentRows(numRows); - queryMetrics.reportPreFilteredRows(preFilteredRows); - } - - return new FilterAnalysis(preFilterBitmap, Filters.maybeAnd(postFilters).orElse(null)); - } - - private final Filter postFilter; - private final ImmutableBitmap preFilterBitmap; - - public FilterAnalysis( - @Nullable final ImmutableBitmap preFilterBitmap, - @Nullable final Filter postFilter - ) - { - this.preFilterBitmap = preFilterBitmap; - this.postFilter = postFilter; - } - - @Nullable - public ImmutableBitmap getPreFilterBitmap() - { - return preFilterBitmap; - } - - @Nullable - public Filter getPostFilter() - { - return postFilter; - } -} diff --git a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java index e7c9f96c65b..755a399f8e0 100644 --- a/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/FilteredOffset.java @@ -19,59 +19,23 @@ package org.apache.druid.segment; -import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.BaseQuery; -import org.apache.druid.query.DefaultBitmapResultFactory; -import org.apache.druid.query.filter.BooleanFilter; import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.filter.RowOffsetMatcherFactory; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.data.Offset; import org.apache.druid.segment.data.ReadableOffset; -import org.apache.druid.segment.filter.ValueMatchers; -import org.apache.druid.segment.index.BitmapColumnIndex; -import org.roaringbitmap.IntIterator; public final class FilteredOffset extends Offset { private final Offset baseOffset; private final ValueMatcher filterMatcher; - FilteredOffset( - Offset baseOffset, - ColumnSelectorFactory columnSelectorFactory, - boolean descending, - Filter postFilter, - ColumnSelectorColumnIndexSelector bitmapIndexSelector - ) + FilteredOffset(Offset baseOffset, ValueMatcher filterMatcher) { this.baseOffset = baseOffset; - RowOffsetMatcherFactory rowOffsetMatcherFactory = new CursorOffsetHolderRowOffsetMatcherFactory( - baseOffset.getBaseReadableOffset(), - descending - ); - if (postFilter instanceof BooleanFilter) { - filterMatcher = ((BooleanFilter) postFilter).makeMatcher( - bitmapIndexSelector, - columnSelectorFactory, - rowOffsetMatcherFactory - ); - } else { - final BitmapColumnIndex columnIndex = postFilter.getBitmapColumnIndex(bitmapIndexSelector); - // we only consider "exact" indexes here, because if false, we've already used the bitmap index for the base - // offset and must use the value matcher here - if (columnIndex != null && columnIndex.getIndexCapabilities().isExact()) { - filterMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher( - columnIndex.computeBitmapResult( - new DefaultBitmapResultFactory(bitmapIndexSelector.getBitmapFactory()), - false - ) - ); - } else { - filterMatcher = postFilter.makeMatcher(columnSelectorFactory); - } - } + this.filterMatcher = filterMatcher; + incrementIfNeededOnCreationOrReset(); } @@ -146,78 +110,4 @@ public final class FilteredOffset extends Offset inspector.visit("baseOffset", baseOffset); inspector.visit("filterMatcher", filterMatcher); } - - private static class CursorOffsetHolderRowOffsetMatcherFactory implements RowOffsetMatcherFactory - { - private final ReadableOffset offset; - private final boolean descending; - - CursorOffsetHolderRowOffsetMatcherFactory(ReadableOffset offset, boolean descending) - { - this.offset = offset; - 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 ValueMatchers.allFalse(); - } - - if (descending) { - return new ValueMatcher() - { - int iterOffset = Integer.MAX_VALUE; - - @Override - public boolean matches(boolean includeUnknown) - { - int currentOffset = offset.getOffset(); - while (iterOffset > currentOffset && iter.hasNext()) { - iterOffset = iter.next(); - } - - return iterOffset == currentOffset; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("offset", offset); - inspector.visit("iter", iter); - } - }; - } else { - return new ValueMatcher() - { - int iterOffset = -1; - - @Override - public boolean matches(boolean includeUnknown) - { - int currentOffset = offset.getOffset(); - while (iterOffset < currentOffset && iter.hasNext()) { - iterOffset = iter.next(); - } - - return iterOffset == currentOffset; - } - - @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) - { - inspector.visit("offset", offset); - inspector.visit("iter", iter); - } - }; - } - } - } } diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java index d6143dfbe37..77bb40f946b 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexCursorSequenceBuilder.java @@ -24,15 +24,21 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.druid.collections.bitmap.ImmutableBitmap; +import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.io.Closer; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.BaseQuery; +import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.Query; import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.filter.Filter; +import org.apache.druid.query.filter.FilterBundle; +import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.NumericColumn; @@ -51,13 +57,17 @@ import org.joda.time.Interval; import javax.annotation.Nullable; import java.io.IOException; +import java.util.concurrent.TimeUnit; public class QueryableIndexCursorSequenceBuilder { + private static final Logger log = new Logger(QueryableIndexCursorSequenceBuilder.class); private final QueryableIndex index; private final Interval interval; private final VirtualColumns virtualColumns; + @Nullable private final Filter filter; + @Nullable private final QueryMetrics metrics; private final long minDataTimestamp; private final long maxDataTimestamp; @@ -99,22 +109,15 @@ public class QueryableIndexCursorSequenceBuilder columnCache ); - final FilterAnalysis filterAnalysis = FilterAnalysis.analyzeFilter( - filter, - bitmapIndexSelector, - metrics, - index.getNumRows() - ); + final int numRows = index.getNumRows(); + final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); - final ImmutableBitmap filterBitmap = filterAnalysis.getPreFilterBitmap(); - final Filter postFilter = filterAnalysis.getPostFilter(); - - if (filterBitmap == null) { - baseOffset = descending - ? new SimpleDescendingOffset(index.getNumRows()) - : new SimpleAscendingOffset(index.getNumRows()); + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter + // can use an index + if (filterBundle == null || filterBundle.getIndex() == null) { + baseOffset = descending ? new SimpleDescendingOffset(numRows) : new SimpleAscendingOffset(numRows); } else { - baseOffset = BitmapOffset.of(filterBitmap, descending, index.getNumRows()); + baseOffset = BitmapOffset.of(filterBundle.getIndex().getBitmap(), descending, index.getNumRows()); } final NumericColumn timestamps = (NumericColumn) columnCache.getColumn(ColumnHolder.TIME_COLUMN_NAME); @@ -175,20 +178,20 @@ public class QueryableIndexCursorSequenceBuilder columnCache ); final DateTime myBucket = gran.toDateTime(inputInterval.getStartMillis()); - - if (postFilter == null) { - return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, myBucket); - } else { - FilteredOffset filteredOffset = new FilteredOffset( - baseCursorOffset, - columnSelectorFactory, - descending, - postFilter, - bitmapIndexSelector - ); + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter + // needs to use a value matcher + if (filterBundle != null && filterBundle.getMatcherBundle() != null) { + final ValueMatcher matcher = filterBundle.getMatcherBundle() + .valueMatcher( + columnSelectorFactory, + baseCursorOffset, + descending + ); + final FilteredOffset filteredOffset = new FilteredOffset(baseCursorOffset, matcher); return new QueryableIndexCursor(filteredOffset, columnSelectorFactory, myBucket); + } else { + return new QueryableIndexCursor(baseCursorOffset, columnSelectorFactory, myBucket); } - } } ), @@ -210,16 +213,8 @@ public class QueryableIndexCursorSequenceBuilder columnCache ); - final FilterAnalysis filterAnalysis = FilterAnalysis.analyzeFilter( - filter, - bitmapIndexSelector, - metrics, - index.getNumRows() - ); - - final ImmutableBitmap filterBitmap = filterAnalysis.getPreFilterBitmap(); - final Filter postFilter = filterAnalysis.getPostFilter(); - + final int numRows = index.getNumRows(); + final FilterBundle filterBundle = makeFilterBundle(bitmapIndexSelector, numRows); NumericColumn timestamps = null; @@ -244,23 +239,27 @@ public class QueryableIndexCursorSequenceBuilder endOffset = index.getNumRows(); } + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter can use + // an index final VectorOffset baseOffset = - filterBitmap == null + filterBundle == null || filterBundle.getIndex() == null ? new NoFilterVectorOffset(vectorSize, startOffset, endOffset) - : new BitmapVectorOffset(vectorSize, filterBitmap, startOffset, endOffset); + : new BitmapVectorOffset(vectorSize, filterBundle.getIndex().getBitmap(), startOffset, endOffset); // baseColumnSelectorFactory using baseOffset is the column selector for filtering. final VectorColumnSelectorFactory baseColumnSelectorFactory = makeVectorColumnSelectorFactoryForOffset( columnCache, baseOffset ); - if (postFilter == null) { - return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize, closer); - } else { + + // filterBundle will only be null if the filter itself is null, otherwise check to see if the filter needs to use + // a value matcher + if (filterBundle != null && filterBundle.getMatcherBundle() != null) { + final VectorValueMatcher vectorValueMatcher = filterBundle.getMatcherBundle() + .vectorMatcher(baseColumnSelectorFactory, baseOffset); final VectorOffset filteredOffset = FilteredVectorOffset.create( baseOffset, - baseColumnSelectorFactory, - postFilter + vectorValueMatcher ); // Now create the cursor and column selector that will be returned to the caller. @@ -269,9 +268,55 @@ public class QueryableIndexCursorSequenceBuilder filteredOffset ); return new QueryableIndexVectorCursor(filteredColumnSelectorFactory, filteredOffset, vectorSize, closer); + } else { + return new QueryableIndexVectorCursor(baseColumnSelectorFactory, baseOffset, vectorSize, closer); } } + @Nullable + private FilterBundle makeFilterBundle( + ColumnSelectorColumnIndexSelector bitmapIndexSelector, + int numRows + ) + { + final BitmapFactory bitmapFactory = bitmapIndexSelector.getBitmapFactory(); + final BitmapResultFactory bitmapResultFactory; + if (metrics != null) { + bitmapResultFactory = metrics.makeBitmapResultFactory(bitmapFactory); + metrics.reportSegmentRows(numRows); + } else { + bitmapResultFactory = new DefaultBitmapResultFactory(bitmapFactory); + } + if (filter == null) { + return null; + } + final long bitmapConstructionStartNs = System.nanoTime(); + final FilterBundle filterBundle = filter.makeFilterBundle( + bitmapIndexSelector, + bitmapResultFactory, + numRows, + numRows, + false + ); + if (metrics != null) { + final long buildTime = System.nanoTime() - bitmapConstructionStartNs; + metrics.reportBitmapConstructionTime(buildTime); + final FilterBundle.BundleInfo info = filterBundle.getInfo(); + metrics.filterBundle(info); + log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); + if (filterBundle.getIndex() != null) { + metrics.reportPreFilteredRows(filterBundle.getIndex().getBitmap().size()); + } else { + metrics.reportPreFilteredRows(0); + } + } else if (log.isDebugEnabled()) { + final FilterBundle.BundleInfo info = filterBundle.getInfo(); + final long buildTime = System.nanoTime() - bitmapConstructionStartNs; + log.debug("Filter partitioning (%sms):%s", TimeUnit.NANOSECONDS.toMillis(buildTime), info); + } + return filterBundle; + } + VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset( ColumnCache columnCache, VectorOffset baseOffset diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java index ead7900c6a8..57c7da95360 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexStorageAdapter.java @@ -19,7 +19,6 @@ package org.apache.druid.segment; -import com.google.common.annotations.VisibleForTesting; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.granularity.Granularities; @@ -187,9 +186,12 @@ public class QueryableIndexStorageAdapter implements StorageAdapter ) { if (filter != null) { - final boolean filterCanVectorize = - filter.getBitmapColumnIndex(makeBitmapIndexSelector(virtualColumns)) != null - || filter.canVectorizeMatcher(this); + // ideally we would allow stuff to vectorize if we can build indexes even if the value matcher cannot be + // vectorized, this used to be true in fact, but changes to filter partitioning (FilterBundle) have caused + // the only way to know this to be building the bitmaps since BitmapColumnIndex can return null. + // this will be changed in a future refactor of cursor building, at which point this method can just return + // true if !descending... + final boolean filterCanVectorize = filter.canVectorizeMatcher(this); if (!filterCanVectorize) { return false; @@ -300,16 +302,4 @@ public class QueryableIndexStorageAdapter implements StorageAdapter return interval.overlap(dataInterval); } - - @VisibleForTesting - public ColumnSelectorColumnIndexSelector makeBitmapIndexSelector( - final VirtualColumns virtualColumns - ) - { - return new ColumnSelectorColumnIndexSelector( - index.getBitmapFactoryForDimensions(), - virtualColumns, - new DeprecatedQueryableIndexColumnSelector(index) - ); - } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java index 62934d3d342..a4bdfcfd6fa 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnConfig.java @@ -19,25 +19,23 @@ package org.apache.druid.segment.column; -import org.apache.druid.segment.index.BitmapColumnIndex; -import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; -import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; import org.apache.druid.segment.index.semantic.NumericRangeIndexes; public interface ColumnConfig { /** - * this value was chosen testing bound filters on double columns with a variety of ranges at which this ratio + * this value was chosen testing range filters on double columns with a variety of ranges at which this ratio * of number of bitmaps compared to total number of rows appeared to be around the threshold where indexes stopped * performing consistently faster than a full scan + value matcher */ - double DEFAULT_SKIP_VALUE_RANGE_INDEX_SCALE = 0.08; - - double DEFAULT_SKIP_VALUE_PREDICATE_INDEX_SCALE = 0.08; + double DEFAULT_SKIP_VALUE_RANGE_INDEX_SCALE = 0.1; ColumnConfig DEFAULT = new ColumnConfig() {}; - ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() + /** + * Use range indexes if dictionary range is same size or smaller than selection size + */ + ColumnConfig SELECTION_SIZE = new ColumnConfig() { @Override @@ -45,20 +43,14 @@ public interface ColumnConfig { return 1.0; } - - @Override - public double skipValuePredicateIndexScale() - { - return 1.0; - } }; /** - * If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap - * index operations required to perform to use {@link LexicographicalRangeIndexes} or {@link NumericRangeIndexes}, - * then for any {@link ColumnIndexSupplier} which chooses to participate in this config it will skip computing the - * index, indicated by a return value of null from the 'forRange' methods, to force the filter to be processed - * with a scan using a {@link org.apache.druid.query.filter.ValueMatcher} instead. + * If the number of selected rows in a column multiplied by this value is smaller than the total number of bitmap + * index operations required to perform to use {@link NumericRangeIndexes}, then for any {@link ColumnIndexSupplier} + * which chooses to participate in this config it will skip computing the index, indicated by a return value of null + * from the 'forRange' methods, to force the filter to be processed with a scan using a + * {@link org.apache.druid.query.filter.ValueMatcher} instead. *

* For range indexes on columns where every value has an index, the number of bitmap operations is determined by how * many individual values fall in the range, a subset of the columns total cardinality. @@ -82,37 +74,4 @@ public interface ColumnConfig { return DEFAULT_SKIP_VALUE_RANGE_INDEX_SCALE; } - - /** - * If the total number of rows in a column multiplied by this value is smaller than the total number of bitmap - * index operations required to perform to use {@link DruidPredicateIndexes} then for any {@link ColumnIndexSupplier} - * which chooses to participate in this config it will skip computing the index, in favor of doing a full scan and - * using a {@link org.apache.druid.query.filter.ValueMatcher} instead. This is indicated returning null from - * {@link ColumnIndexSupplier#as(Class)} even though it would have otherwise been able to create a - * {@link BitmapColumnIndex}. For predicate indexes, this is determined by the total value cardinality of the column - * for columns with an index for every value. - *

- * Currently only the {@link org.apache.druid.segment.nested.NestedCommonFormatColumn} implementations of - * {@link ColumnIndexSupplier} support this behavior. - *

- * This can make some standalone filters faster in cases where the overhead of walking the value dictionary and - * combining bitmaps to construct a {@link org.apache.druid.segment.BitmapOffset} or - * {@link org.apache.druid.segment.vector.BitmapVectorOffset} can exceed the cost of just using doing a full scan - * and using a {@link org.apache.druid.query.filter.ValueMatcher}. - *

- * Where this is especially useful is in cases where the predicate index is used as part of some - * {@link org.apache.druid.segment.filter.AndFilter}, which segment processing partitions into groups of 'pre' - * filters, composed of those which should use indexes, and 'post' filters, which should use a matcher on the offset - * created by the indexes to filter the remaining results. This value pushes what would have been expensive index - * computations to go into the 'pre' group into using a value matcher as part of the 'post' group instead, sometimes - * providing an order of magnitude or higher performance increase. - *

- * This value is separate from {@link #skipValueRangeIndexScale()} since the dynamics of computing predicate indexes - * is potentially different than the much cheaper range calculations (especially for numeric values), so having a - * separate control knob allows for corrections to be done to tune things separately from ranges. - */ - default double skipValuePredicateIndexScale() - { - return DEFAULT_SKIP_VALUE_PREDICATE_INDEX_SCALE; - } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnIndexSupplier.java index 4534745b9ef..7e2b21a25d0 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnIndexSupplier.java @@ -44,14 +44,4 @@ public interface ColumnIndexSupplier */ @Nullable T as(Class clazz); - - static boolean skipComputingRangeIndexes(ColumnConfig columnConfig, int numRows, int rangeSize) - { - return rangeSize > (int) Math.ceil(columnConfig.skipValueRangeIndexScale() * numRows); - } - - static boolean skipComputingPredicateIndexes(ColumnConfig columnConfig, int numRowsToScan, int dictionaryCardinality) - { - return dictionaryCardinality > (int) Math.ceil(columnConfig.skipValuePredicateIndexScale() * numRowsToScan); - } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java index 69884c527a1..f9001689d4a 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/AndFilter.java @@ -23,15 +23,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.BitmapResultFactory; -import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.filter.BooleanFilter; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.filter.RowOffsetMatcherFactory; +import org.apache.druid.query.filter.FilterBundle; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.BaseVectorValueMatcher; import org.apache.druid.query.filter.vector.ReadableVectorMatch; @@ -41,12 +39,13 @@ import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; +import org.apache.druid.segment.data.Offset; import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; import java.util.ArrayList; -import java.util.Collection; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -57,7 +56,7 @@ import java.util.Objects; */ public class AndFilter implements BooleanFilter { - private static final Joiner AND_JOINER = Joiner.on(" && "); + public static final Joiner AND_JOINER = Joiner.on(" && "); private final LinkedHashSet filters; @@ -73,21 +72,136 @@ public class AndFilter implements BooleanFilter this(new LinkedHashSet<>(filters)); } - public static ImmutableBitmap getBitmapIndex( - ColumnIndexSelector selector, + @Override + public FilterBundle makeFilterBundle( + ColumnIndexSelector columnIndexSelector, BitmapResultFactory bitmapResultFactory, - List filters + int selectionRowCount, + int totalRowCount, + boolean includeUnknown ) { - return bitmapResultFactory.toImmutableBitmap( - getBitmapIndex(selector, filters).computeBitmapResult(bitmapResultFactory, false) + final List indexBundleInfos = new ArrayList<>(); + final List matcherBundles = new ArrayList<>(); + final List matcherBundleInfos = new ArrayList<>(); + + int selectionCount = selectionRowCount; + ImmutableBitmap index = null; + ColumnIndexCapabilities merged = new SimpleColumnIndexCapabilities(true, true); + // AND filter can be partitioned into a bundle that has both indexes and value matchers. The filters which support + // indexes are computed into bitmaps and intersected together incrementally, feeding forward the selected row count + // (number of set bits on the bitmap), allowing filters to skip index computation if it would be more expensive + // than using a ValueMatcher for the remaining number of rows. Any filter which was not computed into an index is + // converted into a ValueMatcher which are combined using the makeMatcher/makeVectorMatcher functions. + // We delegate to the makeFilterBundle of the AND clauses to allow this partitioning to be pushed down. For example, + // a nested AND filter might also partition itself into indexes and bundles, and since it is part of a logical AND + // operation, this is valid (and even preferable). + final long bitmapConstructionStartNs = System.nanoTime(); + for (Filter subfilter : filters) { + final FilterBundle subBundle = subfilter.makeFilterBundle( + columnIndexSelector, + bitmapResultFactory, + selectionCount, + totalRowCount, + includeUnknown + ); + if (subBundle.getIndex() != null) { + if (subBundle.getIndex().getBitmap().isEmpty()) { + // if nothing matches for any sub filter, short-circuit, because nothing can possibly match + return FilterBundle.allFalse( + System.nanoTime() - bitmapConstructionStartNs, + columnIndexSelector.getBitmapFactory().makeEmptyImmutableBitmap() + ); + } + merged = merged.merge(subBundle.getIndex().getIndexCapabilities()); + indexBundleInfos.add(subBundle.getIndex().getIndexInfo()); + if (index == null) { + index = subBundle.getIndex().getBitmap(); + } else { + index = index.intersection(subBundle.getIndex().getBitmap()); + } + selectionCount = index.size(); + } + if (subBundle.getMatcherBundle() != null) { + matcherBundles.add(subBundle.getMatcherBundle()); + matcherBundleInfos.add(subBundle.getMatcherBundle().getMatcherInfo()); + } + } + + final FilterBundle.IndexBundle indexBundle; + if (index != null) { + if (indexBundleInfos.size() == 1) { + indexBundle = new FilterBundle.SimpleIndexBundle( + indexBundleInfos.get(0), + index, + merged + ); + } else { + indexBundle = new FilterBundle.SimpleIndexBundle( + new FilterBundle.IndexBundleInfo( + () -> "AND", + selectionCount, + System.nanoTime() - bitmapConstructionStartNs, + indexBundleInfos + ), + index, + merged + ); + } + } else { + indexBundle = null; + } + + final FilterBundle.MatcherBundle matcherBundle; + if (!matcherBundles.isEmpty()) { + matcherBundle = new FilterBundle.MatcherBundle() + { + @Override + public FilterBundle.MatcherBundleInfo getMatcherInfo() + { + if (matcherBundles.size() == 1) { + return matcherBundleInfos.get(0); + } + return new FilterBundle.MatcherBundleInfo( + () -> "AND", + null, + matcherBundleInfos + ); + } + + @Override + public ValueMatcher valueMatcher(ColumnSelectorFactory selectorFactory, Offset baseOffset, boolean descending) + { + final ValueMatcher[] matchers = new ValueMatcher[matcherBundles.size()]; + for (int i = 0; i < matcherBundles.size(); i++) { + matchers[i] = matcherBundles.get(i).valueMatcher(selectorFactory, baseOffset, descending); + } + return makeMatcher(matchers); + } + + @Override + public VectorValueMatcher vectorMatcher(VectorColumnSelectorFactory selectorFactory, ReadableVectorOffset baseOffset) + { + final VectorValueMatcher[] vectorMatchers = new VectorValueMatcher[matcherBundles.size()]; + for (int i = 0; i < matcherBundles.size(); i++) { + vectorMatchers[i] = matcherBundles.get(i).vectorMatcher(selectorFactory, baseOffset); + } + return makeVectorMatcher(vectorMatchers); + } + }; + } else { + matcherBundle = null; + } + + return new FilterBundle( + indexBundle, + matcherBundle ); } - private static BitmapColumnIndex getBitmapIndex( - ColumnIndexSelector selector, - Collection filters - ) + @Nullable + @Override + public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) { if (filters.size() == 1) { return Iterables.getOnlyElement(filters).getBitmapColumnIndex(selector); @@ -122,9 +236,37 @@ public class AndFilter implements BooleanFilter final T bitmapResult = index.computeBitmapResult(bitmapResultFactory, includeUnknown); if (bitmapResultFactory.isEmpty(bitmapResult)) { // Short-circuit. - return bitmapResultFactory.wrapAllFalse( - selector.getBitmapFactory().makeEmptyImmutableBitmap() - ); + return bitmapResultFactory.wrapAllFalse(selector.getBitmapFactory().makeEmptyImmutableBitmap()); + } + bitmapResults.add(bitmapResult); + } + return bitmapResultFactory.intersection(bitmapResults); + } + + @Nullable + @Override + public T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + final List bitmapResults = new ArrayList<>(bitmapColumnIndices.size()); + for (final BitmapColumnIndex index : bitmapColumnIndices) { + final T bitmapResult = index.computeBitmapResult( + bitmapResultFactory, + selectionRowCount, + totalRowCount, + includeUnknown + ); + if (bitmapResult == null) { + // all or nothing + return null; + } + if (bitmapResultFactory.isEmpty(bitmapResult)) { + // Short-circuit. + return bitmapResultFactory.wrapAllFalse(selector.getBitmapFactory().makeEmptyImmutableBitmap()); } bitmapResults.add(bitmapResult); } @@ -133,13 +275,6 @@ public class AndFilter implements BooleanFilter }; } - @Nullable - @Override - public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) - { - return getBitmapIndex(selector, filters); - } - @Override public ValueMatcher makeMatcher(ColumnSelectorFactory factory) { @@ -170,37 +305,6 @@ public class AndFilter implements BooleanFilter return filters.stream().allMatch(filter -> filter.canVectorizeMatcher(inspector)); } - @Override - public ValueMatcher makeMatcher( - ColumnIndexSelector selector, - ColumnSelectorFactory columnSelectorFactory, - RowOffsetMatcherFactory rowOffsetMatcherFactory - ) - { - final List matchers = new ArrayList<>(); - final List bitmaps = new ArrayList<>(); - final BitmapFactory bitmapFactory = selector.getBitmapFactory(); - final DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapFactory); - - for (Filter filter : filters) { - final BitmapColumnIndex columnIndex = filter.getBitmapColumnIndex(selector); - if (columnIndex != null && columnIndex.getIndexCapabilities().isExact()) { - bitmaps.add(columnIndex.computeBitmapResult(resultFactory, false)); - } else { - ValueMatcher matcher = filter.makeMatcher(columnSelectorFactory); - matchers.add(matcher); - } - } - - if (bitmaps.size() > 0) { - ImmutableBitmap combinedBitmap = selector.getBitmapFactory().intersection(bitmaps); - ValueMatcher offsetMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher(combinedBitmap); - matchers.add(0, offsetMatcher); - } - - return makeMatcher(matchers.toArray(BooleanFilter.EMPTY_VALUE_MATCHER_ARRAY)); - } - @Override public LinkedHashSet getFilters() { @@ -235,7 +339,7 @@ public class AndFilter implements BooleanFilter return StringUtils.format("(%s)", AND_JOINER.join(filters)); } - private static ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers) + public static ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers) { Preconditions.checkState(baseMatchers.length > 0); if (baseMatchers.length == 1) { @@ -266,7 +370,7 @@ public class AndFilter implements BooleanFilter }; } - private static VectorValueMatcher makeVectorMatcher(final VectorValueMatcher[] baseMatchers) + public static VectorValueMatcher makeVectorMatcher(final VectorValueMatcher[] baseMatchers) { Preconditions.checkState(baseMatchers.length > 0); if (baseMatchers.length == 1) { @@ -285,7 +389,6 @@ public class AndFilter implements BooleanFilter // Short-circuit if the entire vector is false. break; } - match = matcher.match(match, includeUnknown); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java index f967be191c7..ccd53e96bc2 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/ExpressionFilter.java @@ -30,6 +30,7 @@ import org.apache.druid.math.expr.ExprEval; import org.apache.druid.math.expr.ExpressionType; import org.apache.druid.math.expr.InputBindings; import org.apache.druid.query.filter.ColumnIndexSelector; +import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.DruidDoublePredicate; import org.apache.druid.query.filter.DruidFloatPredicate; import org.apache.druid.query.filter.DruidLongPredicate; @@ -274,10 +275,9 @@ public class ExpressionFilter implements Filter @Override public String toString() { - return "ExpressionFilter{" + - "expr=" + expr + - ", filterTuning=" + filterTuning + - '}'; + return new DimFilter.DimFilterToStringBuilder().append(expr.get().stringify()) + .appendFilterTuning(filterTuning) + .build(); } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/IsBooleanFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/IsBooleanFilter.java index 89eccb7055d..39a4f90c263 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/IsBooleanFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/IsBooleanFilter.java @@ -89,6 +89,38 @@ public class IsBooleanFilter implements Filter selector.getNumRows() ); } + + @Nullable + @Override + public T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + if (isTrue) { + return baseIndex.computeBitmapResult( + bitmapResultFactory, + selectionRowCount, + totalRowCount, + false + ); + } + + final T result = baseIndex.computeBitmapResult( + bitmapResultFactory, + selectionRowCount, + totalRowCount, + useThreeValueLogic + ); + + if (result == null) { + return null; + } + + return bitmapResultFactory.complement(result, selector.getNumRows()); + } }; } return null; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java index fc6440d83cc..eed57784524 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/JavaScriptFilter.java @@ -25,8 +25,13 @@ import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.JavaScriptDimFilter; import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; import java.util.Objects; @@ -66,6 +71,22 @@ public class JavaScriptFilter implements Filter return Filters.makeValueMatcher(factory, dimension, predicateFactory); } + @Override + public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) + { + return ColumnProcessors.makeVectorProcessor( + dimension, + VectorValueMatcherColumnProcessorFactory.instance(), + factory + ).makeMatcher(predicateFactory); + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return true; + } + @Override public Set getRequiredColumns() { diff --git a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java index 0a6fe9a2f0d..3cf415a8d1c 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/LikeFilter.java @@ -24,6 +24,7 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.java.util.common.IAE; import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.filter.ColumnIndexSelector; +import org.apache.druid.query.filter.DimFilter; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.LikeDimFilter; @@ -202,4 +203,15 @@ public class LikeFilter implements Filter { return Objects.hash(dimension, extractionFn, likeMatcher, filterTuning); } + + @Override + public String toString() + { + return new DimFilter.DimFilterToStringBuilder().appendDimension(dimension, extractionFn) + .append(" LIKE '") + .append(likeMatcher) + .append("'") + .appendFilterTuning(filterTuning) + .build(); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java index 1ef0e3c97f7..b4fb793deb1 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/NotFilter.java @@ -88,6 +88,27 @@ public class NotFilter implements Filter selector.getNumRows() ); } + + @Nullable + @Override + public T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + final T result = baseIndex.computeBitmapResult( + bitmapResultFactory, + selectionRowCount, + totalRowCount, + !includeUnknown && useThreeValueLogic + ); + if (result == null) { + return null; + } + return bitmapResultFactory.complement(result, selector.getNumRows()); + } }; } return null; diff --git a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java index e03bf034482..9373ea142e9 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/OrFilter.java @@ -22,30 +22,36 @@ package org.apache.druid.segment.filter; import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import org.apache.druid.collections.bitmap.BitmapFactory; +import com.google.common.collect.Lists; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.BitmapResultFactory; -import org.apache.druid.query.DefaultBitmapResultFactory; import org.apache.druid.query.filter.BooleanFilter; import org.apache.druid.query.filter.ColumnIndexSelector; import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.filter.RowOffsetMatcherFactory; +import org.apache.druid.query.filter.FilterBundle; import org.apache.druid.query.filter.ValueMatcher; import org.apache.druid.query.filter.vector.BaseVectorValueMatcher; +import org.apache.druid.query.filter.vector.BooleanVectorValueMatcher; import org.apache.druid.query.filter.vector.ReadableVectorMatch; import org.apache.druid.query.filter.vector.VectorMatch; import org.apache.druid.query.filter.vector.VectorValueMatcher; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.BitmapOffset; import org.apache.druid.segment.ColumnInspector; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.SimpleColumnIndexCapabilities; +import org.apache.druid.segment.data.Offset; +import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorColumnSelectorFactory; +import org.roaringbitmap.IntIterator; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedHashSet; import java.util.List; import java.util.Map; @@ -71,6 +77,172 @@ public class OrFilter implements BooleanFilter this(new LinkedHashSet<>(filters)); } + @Override + public FilterBundle makeFilterBundle( + ColumnIndexSelector columnIndexSelector, + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + // for OR filters, we have a few possible outcomes: + // 1 - all clauses are index only bundles. in this case we union the bitmaps together and make an index only bundle + // 2 - some clauses support indexes. in this case, we union the bitmaps of any index only bundles together to form a + // partial index which is constructed into a matcher bundle with convertIndexToMatcherBundle. We translate any + // index AND matcher bundles into a matcher only bundle with convertBundleToMatcherOnlyBundle. Finally, we + // combine these with the remaining matcher only bundles to with makeMatcher/makeVectorMatcher to make a matcher + // only bundle + // 3 - no clauses support indexes. in this case, we make a matcher only bundle using makeMatcher/makeVectorMatcher + + final List indexOnlyBundles = new ArrayList<>(); + final List indexOnlyBundlesInfo = new ArrayList<>(); + final List partialIndexBundles = new ArrayList<>(); + final List matcherOnlyBundles = new ArrayList<>(); + ImmutableBitmap index = null; + ColumnIndexCapabilities merged = new SimpleColumnIndexCapabilities(true, true); + int emptyCount = 0; + + final long bitmapConstructionStartNs = System.nanoTime(); + + for (Filter subfilter : filters) { + final FilterBundle bundle = subfilter.makeFilterBundle( + columnIndexSelector, + bitmapResultFactory, + selectionRowCount, + totalRowCount, + includeUnknown + ); + if (bundle.hasIndex()) { + final ImmutableBitmap bundleIndex = bundle.getIndex().getBitmap(); + if (bundleIndex.isEmpty()) { + // we leave any indexes which are empty out of index, indexOnlyBundles, and partialIndexBundles + // even though we skip them, we still keep track of them to check for the case when we can build the OR into + // an index only bundle. We can count index and matcher bundles here too because the AND operation means that + // an empty index means the matcher can be skipped + emptyCount++; + } else { + if (bundle.hasMatcher()) { + // index and matcher bundles must be handled separately, they will need to be a single value matcher built + // by doing an AND operation between the index and the value matcher + // (a bundle is basically an AND operation between the index and matcher if the matcher is present) + partialIndexBundles.add(convertBundleToMatcherOnlyBundle(bundle, bundleIndex)); + } else { + indexOnlyBundles.add(bundle.getIndex()); + indexOnlyBundlesInfo.add(bundle.getIndex().getIndexInfo()); + merged.merge(bundle.getIndex().getIndexCapabilities()); + // union index only bitmaps together; if all sub-filters are 'index only' bundles we will make an index only + // bundle ourselves, else we will use this index as a single value matcher + if (index == null) { + index = bundle.getIndex().getBitmap(); + } else { + index = index.union(bundle.getIndex().getBitmap()); + } + } + } + } else { + matcherOnlyBundles.add(bundle.getMatcherBundle()); + } + } + final long totalBitmapConstructTimeNs = System.nanoTime() - bitmapConstructionStartNs; + + + // if all the filters are 'index only', we can make an index only bundle + if (indexOnlyBundles.size() + emptyCount == filters.size()) { + if (index == null || index.isEmpty()) { + return FilterBundle.allFalse( + totalBitmapConstructTimeNs, + columnIndexSelector.getBitmapFactory().makeEmptyImmutableBitmap() + ); + } + if (indexOnlyBundles.size() == 1) { + return new FilterBundle( + indexOnlyBundles.get(0), + null + ); + } + return new FilterBundle( + new FilterBundle.SimpleIndexBundle( + new FilterBundle.IndexBundleInfo( + () -> "OR", + selectionRowCount, + totalBitmapConstructTimeNs, + indexOnlyBundlesInfo + ), + index, + merged + ), + null + ); + } + + // if not the index only outcome, we build a matcher only bundle from all the matchers + final int estimatedSize = (indexOnlyBundles.isEmpty() ? 0 : 1) + + partialIndexBundles.size() + + matcherOnlyBundles.size(); + final List allMatcherBundles = Lists.newArrayListWithCapacity(estimatedSize); + final List allMatcherBundlesInfo = Lists.newArrayListWithCapacity(estimatedSize); + if (!indexOnlyBundles.isEmpty()) { + // translate the indexOnly bundles into a single matcher + final FilterBundle.MatcherBundle matcherBundle = convertIndexToMatcherBundle( + selectionRowCount, + indexOnlyBundles, + indexOnlyBundlesInfo, + totalBitmapConstructTimeNs, + index + ); + allMatcherBundles.add(matcherBundle); + allMatcherBundlesInfo.add(matcherBundle.getMatcherInfo()); + } + for (FilterBundle.MatcherBundle bundle : partialIndexBundles) { + allMatcherBundles.add(bundle); + allMatcherBundlesInfo.add(bundle.getMatcherInfo()); + } + for (FilterBundle.MatcherBundle bundle : matcherOnlyBundles) { + allMatcherBundles.add(bundle); + allMatcherBundlesInfo.add(bundle.getMatcherInfo()); + } + + return new FilterBundle( + null, + new FilterBundle.MatcherBundle() + { + @Override + public FilterBundle.MatcherBundleInfo getMatcherInfo() + { + return new FilterBundle.MatcherBundleInfo( + () -> "OR", + null, + allMatcherBundlesInfo + ); + } + + @Override + public ValueMatcher valueMatcher(ColumnSelectorFactory selectorFactory, Offset baseOffset, boolean descending) + { + final ValueMatcher[] matchers = new ValueMatcher[allMatcherBundles.size()]; + for (int i = 0; i < allMatcherBundles.size(); i++) { + matchers[i] = allMatcherBundles.get(i).valueMatcher(selectorFactory, baseOffset, descending); + } + return makeMatcher(matchers); + } + + @Override + public VectorValueMatcher vectorMatcher( + VectorColumnSelectorFactory selectorFactory, + ReadableVectorOffset baseOffset + ) + { + final VectorValueMatcher[] matchers = new VectorValueMatcher[allMatcherBundles.size()]; + for (int i = 0; i < allMatcherBundles.size(); i++) { + matchers[i] = allMatcherBundles.get(i).vectorMatcher(selectorFactory, baseOffset); + } + return makeVectorMatcher(matchers); + } + } + ); + } + @Nullable @Override public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) @@ -107,6 +279,27 @@ public class OrFilter implements BooleanFilter () -> bitmapColumnIndices.stream().map(x -> x.computeBitmapResult(bitmapResultFactory, includeUnknown)).iterator() ); } + + @Nullable + @Override + public T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + List results = Lists.newArrayListWithCapacity(bitmapColumnIndices.size()); + for (BitmapColumnIndex index : bitmapColumnIndices) { + final T r = index.computeBitmapResult(bitmapResultFactory, selectionRowCount, totalRowCount, includeUnknown); + if (r == null) { + // all or nothing + return null; + } + results.add(r); + } + return bitmapResultFactory.union(results); + } }; } @@ -140,37 +333,6 @@ public class OrFilter implements BooleanFilter return filters.stream().allMatch(filter -> filter.canVectorizeMatcher(inspector)); } - @Override - public ValueMatcher makeMatcher( - ColumnIndexSelector selector, - ColumnSelectorFactory columnSelectorFactory, - RowOffsetMatcherFactory rowOffsetMatcherFactory - ) - { - final List matchers = new ArrayList<>(); - final List bitmaps = new ArrayList<>(); - final BitmapFactory bitmapFactory = selector.getBitmapFactory(); - final DefaultBitmapResultFactory resultFactory = new DefaultBitmapResultFactory(bitmapFactory); - - for (Filter filter : filters) { - final BitmapColumnIndex columnIndex = filter.getBitmapColumnIndex(selector); - if (columnIndex != null && columnIndex.getIndexCapabilities().isExact()) { - bitmaps.add(columnIndex.computeBitmapResult(resultFactory, false)); - } else { - ValueMatcher matcher = filter.makeMatcher(columnSelectorFactory); - matchers.add(matcher); - } - } - - if (bitmaps.size() > 0) { - ImmutableBitmap combinedBitmap = selector.getBitmapFactory().union(bitmaps); - ValueMatcher offsetMatcher = rowOffsetMatcherFactory.makeRowOffsetMatcher(combinedBitmap); - matchers.add(0, offsetMatcher); - } - - return makeMatcher(matchers.toArray(BooleanFilter.EMPTY_VALUE_MATCHER_ARRAY)); - } - @Override public LinkedHashSet getFilters() { @@ -205,6 +367,26 @@ public class OrFilter implements BooleanFilter return StringUtils.format("(%s)", OR_JOINER.join(filters)); } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + OrFilter orFilter = (OrFilter) o; + return Objects.equals(getFilters(), orFilter.getFilters()); + } + + @Override + public int hashCode() + { + return Objects.hash(getFilters()); + } + + private static ValueMatcher makeMatcher(final ValueMatcher[] baseMatchers) { Preconditions.checkState(baseMatchers.length > 0); @@ -286,22 +468,246 @@ public class OrFilter implements BooleanFilter }; } - @Override - public boolean equals(Object o) + /** + * Convert a {@link FilterBundle} that has both {@link FilterBundle#getIndex()} and + * {@link FilterBundle#getMatcherBundle()} into a 'matcher only' bundle by converting the index into a matcher + * with {@link #convertIndexToValueMatcher(ReadableOffset, ImmutableBitmap, boolean)} and + * {@link #convertIndexToVectorValueMatcher(ReadableVectorOffset, ImmutableBitmap)} and then doing a logical AND + * with the bundles matchers. + */ + private static FilterBundle.MatcherBundle convertBundleToMatcherOnlyBundle( + FilterBundle bundle, + ImmutableBitmap bundleIndex + ) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - OrFilter orFilter = (OrFilter) o; - return Objects.equals(getFilters(), orFilter.getFilters()); + return new FilterBundle.MatcherBundle() + { + @Override + public FilterBundle.MatcherBundleInfo getMatcherInfo() + { + return new FilterBundle.MatcherBundleInfo( + () -> "AND", + bundle.getIndex().getIndexInfo(), + Collections.singletonList(bundle.getMatcherBundle().getMatcherInfo()) + ); + } + + @Override + public ValueMatcher valueMatcher( + ColumnSelectorFactory selectorFactory, + Offset baseOffset, + boolean descending + ) + { + return AndFilter.makeMatcher( + new ValueMatcher[]{ + convertIndexToValueMatcher(baseOffset.getBaseReadableOffset(), bundleIndex, descending), + bundle.getMatcherBundle().valueMatcher(selectorFactory, baseOffset, descending) + } + ); + } + + @Override + public VectorValueMatcher vectorMatcher( + VectorColumnSelectorFactory selectorFactory, + ReadableVectorOffset baseOffset + ) + { + return AndFilter.makeVectorMatcher( + new VectorValueMatcher[]{ + convertIndexToVectorValueMatcher( + baseOffset, + bundleIndex + ), + bundle.getMatcherBundle().vectorMatcher(selectorFactory, baseOffset) + } + ); + } + }; } - @Override - public int hashCode() + /** + * Convert an index into a matcher bundle, using + * {@link #convertIndexToValueMatcher(ReadableOffset, ImmutableBitmap, boolean)} and + * {@link #convertIndexToVectorValueMatcher(ReadableVectorOffset, ImmutableBitmap)} + */ + private static FilterBundle.MatcherBundle convertIndexToMatcherBundle( + int selectionRowCount, + List indexOnlyBundles, + List indexOnlyBundlesInfo, + long totalBitmapConstructTimeNs, + ImmutableBitmap partialIndex + ) { - return Objects.hash(getFilters()); + return new FilterBundle.MatcherBundle() + { + @Override + public FilterBundle.MatcherBundleInfo getMatcherInfo() + { + if (indexOnlyBundles.size() == 1) { + return new FilterBundle.MatcherBundleInfo( + indexOnlyBundles.get(0).getIndexInfo()::getFilter, + indexOnlyBundles.get(0).getIndexInfo(), + null + ); + } + return new FilterBundle.MatcherBundleInfo( + () -> "OR", + new FilterBundle.IndexBundleInfo( + () -> "OR", + selectionRowCount, + totalBitmapConstructTimeNs, + indexOnlyBundlesInfo + ), + null + ); + } + + @Override + public ValueMatcher valueMatcher( + ColumnSelectorFactory selectorFactory, + Offset baseOffset, + boolean descending + ) + { + return convertIndexToValueMatcher(baseOffset.getBaseReadableOffset(), partialIndex, descending); + } + + @Override + public VectorValueMatcher vectorMatcher( + VectorColumnSelectorFactory selectorFactory, + ReadableVectorOffset baseOffset + ) + { + return convertIndexToVectorValueMatcher(baseOffset, partialIndex); + } + }; + } + + private static ValueMatcher convertIndexToValueMatcher( + final ReadableOffset offset, + final ImmutableBitmap rowBitmap, + boolean descending + ) + { + final IntIterator iter = descending ? + BitmapOffset.getReverseBitmapOffsetIterator(rowBitmap) : + rowBitmap.iterator(); + + if (!iter.hasNext()) { + return ValueMatchers.allFalse(); + } + + if (descending) { + return new ValueMatcher() + { + int iterOffset = Integer.MAX_VALUE; + + @Override + public boolean matches(boolean includeUnknown) + { + int currentOffset = offset.getOffset(); + while (iterOffset > currentOffset && iter.hasNext()) { + iterOffset = iter.next(); + } + + return iterOffset == currentOffset; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("offset", offset); + inspector.visit("iter", iter); + } + }; + } else { + return new ValueMatcher() + { + int iterOffset = -1; + + @Override + public boolean matches(boolean includeUnknown) + { + int currentOffset = offset.getOffset(); + while (iterOffset < currentOffset && iter.hasNext()) { + iterOffset = iter.next(); + } + + return iterOffset == currentOffset; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + inspector.visit("offset", offset); + inspector.visit("iter", iter); + } + }; + } + } + + private static VectorValueMatcher convertIndexToVectorValueMatcher( + final ReadableVectorOffset vectorOffset, + final ImmutableBitmap bitmap + ) + { + final IntIterator bitmapIterator = bitmap.iterator(); + if (!bitmapIterator.hasNext()) { + return BooleanVectorValueMatcher.of(vectorOffset, ConstantMatcherType.ALL_FALSE); + } + + return new VectorValueMatcher() + { + final VectorMatch match = VectorMatch.wrap(new int[vectorOffset.getMaxVectorSize()]); + int iterOffset = -1; + @Override + public ReadableVectorMatch match(ReadableVectorMatch mask, boolean includeUnknown) + { + final int[] selection = match.getSelection(); + if (vectorOffset.isContiguous()) { + int numRows = 0; + for (int i = 0; i < mask.getSelectionSize(); i++) { + final int maskNum = mask.getSelection()[i]; + final int rowNum = vectorOffset.getStartOffset() + maskNum; + while (iterOffset < rowNum && bitmapIterator.hasNext()) { + iterOffset = bitmapIterator.next(); + } + if (iterOffset == rowNum) { + selection[numRows++] = maskNum; + } + } + match.setSelectionSize(numRows); + return match; + } else { + final int[] currentOffsets = vectorOffset.getOffsets(); + int numRows = 0; + for (int i = 0; i < mask.getSelectionSize(); i++) { + final int maskNum = mask.getSelection()[i]; + final int rowNum = currentOffsets[mask.getSelection()[i]]; + while (iterOffset < rowNum && bitmapIterator.hasNext()) { + iterOffset = bitmapIterator.next(); + } + if (iterOffset == rowNum) { + selection[numRows++] = maskNum; + } + } + match.setSelectionSize(numRows); + return match; + } + } + + @Override + public int getMaxVectorSize() + { + return vectorOffset.getMaxVectorSize(); + } + + @Override + public int getCurrentVectorSize() + { + return vectorOffset.getCurrentVectorSize(); + } + }; } } diff --git a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java index 78a5c6633f3..2d3f88912bb 100644 --- a/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java +++ b/processing/src/main/java/org/apache/druid/segment/filter/SpatialFilter.java @@ -35,6 +35,10 @@ import org.apache.druid.query.filter.DruidPredicateMatch; import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.ValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcher; +import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory; +import org.apache.druid.segment.ColumnInspector; +import org.apache.druid.segment.ColumnProcessors; import org.apache.druid.segment.ColumnSelectorFactory; import org.apache.druid.segment.column.ColumnIndexCapabilities; import org.apache.druid.segment.column.ColumnIndexSupplier; @@ -43,6 +47,7 @@ import org.apache.druid.segment.incremental.SpatialDimensionRowTransformer; import org.apache.druid.segment.index.AllUnknownBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; import org.apache.druid.segment.index.semantic.SpatialIndex; +import org.apache.druid.segment.vector.VectorColumnSelectorFactory; import javax.annotation.Nullable; import java.util.Objects; @@ -106,10 +111,25 @@ public class SpatialFilter implements Filter factory, dimension, new BoundDruidPredicateFactory(bound) - ); } + @Override + public VectorValueMatcher makeVectorMatcher(VectorColumnSelectorFactory factory) + { + return ColumnProcessors.makeVectorProcessor( + dimension, + VectorValueMatcherColumnProcessorFactory.instance(), + factory + ).makeMatcher(new BoundDruidPredicateFactory(bound)); + } + + @Override + public boolean canVectorizeMatcher(ColumnInspector inspector) + { + return true; + } + @Override public Set getRequiredColumns() { diff --git a/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java index a8d10b2ac45..1421e4fc16e 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/BitmapColumnIndex.java @@ -23,6 +23,8 @@ import org.apache.druid.common.config.NullHandling; import org.apache.druid.query.BitmapResultFactory; import org.apache.druid.segment.column.ColumnIndexCapabilities; +import javax.annotation.Nullable; + /** * Common interface for bitmap indexes for use by {@link org.apache.druid.query.filter.Filter} for cursor creation, to * allow fast row skipping during query processing. @@ -33,6 +35,7 @@ public interface BitmapColumnIndex /** * Compute a bitmap result wrapped with the {@link BitmapResultFactory} representing the rows matched by this index. + * If building a cursor, use {@link #computeBitmapResult(BitmapResultFactory, int, int, boolean)} instead. * * @param bitmapResultFactory helper to format the {@link org.apache.druid.collections.bitmap.ImmutableBitmap} in a * form ready for consumption by callers @@ -43,5 +46,34 @@ public interface BitmapColumnIndex * * @return bitmap result representing rows matched by this index */ - T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown); + T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + boolean includeUnknown + ); + + /** + * Compute a bitmap result wrapped with the {@link BitmapResultFactory} representing the rows matched by this index, + * or null if the index cannot (or should not) be computed. + * + * @param bitmapResultFactory helper to format the {@link org.apache.druid.collections.bitmap.ImmutableBitmap} in a + * form ready for consumption by callers + * @param selectionRowCount number of rows selected so far by any previous index computations + * @param totalRowCount total number of rows to be scanned if no indexes are used + * @param includeUnknown mapping for Druid native two state logic system into SQL three-state logic system. If + * set to true, bitmaps returned by this method should include true bits for any rows where + * the matching result is 'unknown', such as from the input being null valued. + * See {@link NullHandling#useThreeValueLogic()}. + * + * @return bitmap result representing rows matched by this index + */ + @Nullable + default T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + return computeBitmapResult(bitmapResultFactory, includeUnknown); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/index/DictionaryRangeScanningBitmapIndex.java b/processing/src/main/java/org/apache/druid/segment/index/DictionaryRangeScanningBitmapIndex.java new file mode 100644 index 00000000000..6b175297181 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/index/DictionaryRangeScanningBitmapIndex.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.index; + +import org.apache.druid.query.BitmapResultFactory; +import org.apache.druid.segment.column.ColumnConfig; + +import javax.annotation.Nullable; + +/** + * Common {@link BitmapColumnIndex} implementation for indexes which need to scan a range of values. Contains logic + * to skip computing indexes with {@link #computeBitmapResult(BitmapResultFactory, int, int, boolean)} if + * {@link #rangeSize} is larger than {@link #sizeScale} multiplied by the number of selected rows. Numeric range + * indexes will typically want to set {@link #sizeScale} to a double closer to 0.0 than to 1.0 because numeric + * comparisons are relatively cheap compared to bitmap operations. Most numerical implementations should use the + * value of {@link ColumnConfig#skipValueRangeIndexScale()}. + *

+ * Other implementations should adjust {@link #sizeScale} as appropriate for the expense of the value matcher compared + * to the expense of the bitmap operations. + */ +public abstract class DictionaryRangeScanningBitmapIndex extends SimpleImmutableBitmapDelegatingIterableIndex +{ + private final double sizeScale; + private final int rangeSize; + + public DictionaryRangeScanningBitmapIndex(double sizeScale, int rangeSize) + { + this.sizeScale = sizeScale; + this.rangeSize = rangeSize; + } + + @Nullable + @Override + public final T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + final int scale = (int) Math.ceil(sizeScale * selectionRowCount); + if (rangeSize > scale) { + return null; + } + return computeBitmapResult(bitmapResultFactory, includeUnknown); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/index/DictionaryScanningBitmapIndex.java b/processing/src/main/java/org/apache/druid/segment/index/DictionaryScanningBitmapIndex.java new file mode 100644 index 00000000000..2bde27dc52c --- /dev/null +++ b/processing/src/main/java/org/apache/druid/segment/index/DictionaryScanningBitmapIndex.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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 org.apache.druid.segment.index; + +import org.apache.druid.query.BitmapResultFactory; + +import javax.annotation.Nullable; + +/** + * Common {@link BitmapColumnIndex} implementation for indexes which are computed by scanning the entire value + * dictionary of the underlying column to check if the value index matches the filter. Contains logic to skip computing + * indexes with {@link #computeBitmapResult(BitmapResultFactory, int, int, boolean)} if 'selectionRowCount' does not + * equal 'totalRowCount' and 'selectionRowCount' is smaller than {@link #dictionarySize} multiplied by + * {@link #scaleThreshold}. The default {@link #scaleThreshold} value is 1.0, meaning that if {@link #dictionarySize} + * is larger than 'selectionRowCount' we skip using indexes, the idea being we would either have to perform the check + * against the values in the dictionary or the values in the remaining rows, since remaining rows is smaller we should + * just do that instead of spending time to compute indexes to further shrink 'selectionRowCount'. + */ +public abstract class DictionaryScanningBitmapIndex extends SimpleImmutableBitmapIterableIndex +{ + private final int dictionarySize; + private final double scaleThreshold; + + public DictionaryScanningBitmapIndex(int dictionarySize) + { + this(dictionarySize, 1.0); + } + + public DictionaryScanningBitmapIndex(int dictionarySize, double scaleThreshold) + { + this.dictionarySize = dictionarySize; + this.scaleThreshold = scaleThreshold; + } + + @Nullable + @Override + public final T computeBitmapResult( + BitmapResultFactory bitmapResultFactory, + int selectionRowCount, + int totalRowCount, + boolean includeUnknown + ) + { + if (selectionRowCount != totalRowCount && selectionRowCount < (dictionarySize * scaleThreshold)) { + return null; + } + return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable(includeUnknown)); + } +} diff --git a/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java index b3076ca7606..8429b8b6a9f 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedStringDruidPredicateIndexes.java @@ -23,8 +23,6 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.query.filter.DruidObjectPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; @@ -38,34 +36,25 @@ public final class IndexedStringDruidPredicateIndexes bitmaps; - private final ColumnConfig columnConfig; - private final int numRows; public IndexedStringDruidPredicateIndexes( BitmapFactory bitmapFactory, TDictionary dictionary, - Indexed bitmaps, - @Nullable ColumnConfig columnConfig, - int numRows + Indexed bitmaps ) { this.bitmapFactory = bitmapFactory; this.dictionary = dictionary; this.bitmaps = bitmaps; - this.columnConfig = columnConfig; - this.numRows = numRows; } @Override @Nullable public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory) { - if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, dictionary.size())) { - return null; - } final DruidObjectPredicate stringPredicate = matcherFactory.makeStringPredicate(); - return new SimpleImmutableBitmapIterableIndex() + return new DictionaryScanningBitmapIndex(dictionary.size()) { @Override public Iterable getBitmapIterable(boolean includeUnknown) diff --git a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java index 3e30589c3d5..bc553b1bd01 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java +++ b/processing/src/main/java/org/apache/druid/segment/index/IndexedUtf8LexicographicalRangeIndexes.java @@ -30,8 +30,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.filter.DruidObjectPredicate; import org.apache.druid.query.filter.DruidPredicateMatch; import org.apache.druid.segment.IntListUtils; -import org.apache.druid.segment.column.ColumnConfig; -import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.data.Indexed; import org.apache.druid.segment.index.semantic.LexicographicalRangeIndexes; @@ -48,16 +46,11 @@ public final class IndexedUtf8LexicographicalRangeIndexes bitmaps; private final boolean hasNull; - private final ColumnConfig columnConfig; - private final int numRows; - public IndexedUtf8LexicographicalRangeIndexes( BitmapFactory bitmapFactory, TDictionary dictionary, Indexed bitmaps, - boolean hasNull, - @Nullable ColumnConfig columnConfig, - int numRows + boolean hasNull ) { Preconditions.checkArgument(dictionary.isSorted(), "Dictionary must be sorted"); @@ -65,8 +58,6 @@ public final class IndexedUtf8LexicographicalRangeIndexes getBitmapIterable() @@ -132,10 +120,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes getBitmapIterable() diff --git a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapDelegatingIterableIndex.java b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapDelegatingIterableIndex.java index 5dceacfafc5..2c0942506cf 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapDelegatingIterableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapDelegatingIterableIndex.java @@ -29,7 +29,7 @@ import java.util.Collections; public abstract class SimpleImmutableBitmapDelegatingIterableIndex extends SimpleBitmapColumnIndex { @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) + public final T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { if (includeUnknown) { final ImmutableBitmap unknownsBitmap = getUnknownsBitmap(); diff --git a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java index 9803be5aa2d..9901d897d7d 100644 --- a/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/index/SimpleImmutableBitmapIterableIndex.java @@ -27,9 +27,8 @@ import org.apache.druid.query.BitmapResultFactory; */ public abstract class SimpleImmutableBitmapIterableIndex extends SimpleBitmapColumnIndex { - @Override - public T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) + public final T computeBitmapResult(BitmapResultFactory bitmapResultFactory, boolean includeUnknown) { return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable(includeUnknown)); } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java index cc259c3be0c..d3869bd9ef5 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java @@ -999,10 +999,6 @@ public abstract class CompressedNestedDataComplexColumn> globalDoubleDictionarySupplier, @Nullable Supplier globalArrayDictionarySupplier, @Nullable Supplier> arrayElementDictionarySupplier, - @Nullable GenericIndexed arrayElementBitmaps, - int numRows + @Nullable GenericIndexed arrayElementBitmaps ) { this.singleType = types.getSingleType(); @@ -144,7 +143,6 @@ public class NestedFieldColumnIndexSupplier localDictionary, Indexed globalDictionary, - int adjust + int adjust, + double rangeScale ) { final IntIntPair localRange = getLocalRangeFromDictionary( @@ -318,10 +317,7 @@ public class NestedFieldColumnIndexSupplier getBitmapIterable() @@ -509,7 +505,8 @@ public class NestedFieldColumnIndexSupplier getBitmapIterable() @@ -606,10 +600,7 @@ public class NestedFieldColumnIndexSupplier localDictionary = localDictionarySupplier.get(); - if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, localDictionary.size())) { - return null; - } - return new SimpleImmutableBitmapIterableIndex() + return new DictionaryScanningBitmapIndex(localDictionary.size()) { @Override public Iterable getBitmapIterable(boolean includeUnknown) @@ -875,7 +866,8 @@ public class NestedFieldColumnIndexSupplier localDictionary = localDictionarySupplier.get(); - if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, localDictionary.size())) { - return null; - } - return new SimpleImmutableBitmapIterableIndex() + return new DictionaryScanningBitmapIndex(localDictionary.size()) { + @Override public Iterable getBitmapIterable(boolean includeUnknown) { @@ -1142,7 +1132,8 @@ public class NestedFieldColumnIndexSupplier localDictionary = localDictionarySupplier.get(); - if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, localDictionary.size())) { - return null; - } - return new SimpleImmutableBitmapIterableIndex() + return new DictionaryScanningBitmapIndex(localDictionary.size()) { @Override public Iterable getBitmapIterable(boolean includeUnknown) @@ -1379,10 +1367,7 @@ public class NestedFieldColumnIndexSupplier getBitmapIterable(boolean includeUnknown) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java index 7cb413077c5..97788148288 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarDoubleColumnAndIndexSupplier.java @@ -55,10 +55,11 @@ import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.VByte; import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.DictionaryRangeScanningBitmapIndex; +import org.apache.druid.segment.index.DictionaryScanningBitmapIndex; import org.apache.druid.segment.index.SimpleBitmapColumnIndex; import org.apache.druid.segment.index.SimpleImmutableBitmapDelegatingIterableIndex; import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; -import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; @@ -128,17 +129,12 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier> longDictionary, Supplier valueColumnSupplier, GenericIndexed valueIndexes, BitmapFactory bitmapFactory, - ColumnConfig columnConfig, - int numRows + ColumnConfig columnConfig ) { this.doubleDictionarySupplier = longDictionary; @@ -175,7 +169,6 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier getBitmapIterable() @@ -467,10 +460,7 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier dictionary = doubleDictionarySupplier.get(); - if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, dictionary.size())) { - return null; - } - return new SimpleImmutableBitmapIterableIndex() + return new DictionaryScanningBitmapIndex(dictionary.size()) { @Override public Iterable getBitmapIterable(boolean includeUnknown) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java index 978b1b1526a..d2aa90e607e 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarLongColumnAndIndexSupplier.java @@ -54,10 +54,11 @@ import org.apache.druid.segment.data.GenericIndexed; import org.apache.druid.segment.data.VByte; import org.apache.druid.segment.index.AllFalseBitmapColumnIndex; import org.apache.druid.segment.index.BitmapColumnIndex; +import org.apache.druid.segment.index.DictionaryRangeScanningBitmapIndex; +import org.apache.druid.segment.index.DictionaryScanningBitmapIndex; import org.apache.druid.segment.index.SimpleBitmapColumnIndex; import org.apache.druid.segment.index.SimpleImmutableBitmapDelegatingIterableIndex; import org.apache.druid.segment.index.SimpleImmutableBitmapIndex; -import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex; import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex; import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex; import org.apache.druid.segment.index.semantic.DruidPredicateIndexes; @@ -127,17 +128,12 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier> longDictionarySupplier, Supplier valueColumnSupplier, GenericIndexed valueIndexes, BitmapFactory bitmapFactory, - ColumnConfig columnConfig, - int numRows + ColumnConfig columnConfig ) { this.longDictionarySupplier = longDictionarySupplier; @@ -175,7 +169,6 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier getBitmapIterable() @@ -477,10 +470,7 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier dictionary = longDictionarySupplier.get(); - if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, dictionary.size())) { - return null; - } - return new SimpleImmutableBitmapIterableIndex() + return new DictionaryScanningBitmapIndex(dictionary.size()) { @Override public Iterable getBitmapIterable(boolean includeUnknown) diff --git a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java index d9ba9ee4e5d..7d957e02a5b 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/ScalarStringColumnAndIndexSupplier.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; import org.apache.druid.segment.column.ColumnBuilder; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.column.StringUtf8DictionaryEncodedColumn; @@ -49,8 +48,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier> dictionarySupplier, Supplier encodedColumnSupplier, GenericIndexed valueIndexes, - BitmapSerdeFactory serdeFactory, - ColumnConfig columnConfig, - int numRows + BitmapSerdeFactory serdeFactory ) { this.dictionarySupplier = dictionarySupplier; @@ -134,9 +124,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier arrayDictionarySupplier; private final Supplier> arrayElementDictionarySupplier; private final Supplier encodedValueColumnSupplier; - @SuppressWarnings("unused") private final GenericIndexed valueIndexes; - @SuppressWarnings("unused") private final GenericIndexed arrayElementIndexes; private final ImmutableBitmap nullValueBitmap; @@ -236,9 +226,7 @@ public class VariantColumnAndIndexSupplier implements Supplier encodedValueColumnSupplier, GenericIndexed valueIndexes, GenericIndexed elementIndexes, - BitmapFactory bitmapFactory, - @SuppressWarnings("unused") ColumnConfig columnConfig, - @SuppressWarnings("unused") int numRows + BitmapFactory bitmapFactory ) { this.logicalType = logicalType; diff --git a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java index 55ec10466e9..a80bebf129f 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/NestedCommonFormatColumnPartSerde.java @@ -196,8 +196,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde byteOrder, bitmapSerdeFactory, buffer, - builder, - columnConfig + builder ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); capabilitiesBuilder.setDictionaryEncoded(true); @@ -264,8 +263,7 @@ public class NestedCommonFormatColumnPartSerde implements ColumnPartSerde byteOrder, bitmapSerdeFactory, buffer, - builder, - columnConfig + builder ); ColumnCapabilitiesImpl capabilitiesBuilder = builder.getCapabilitiesBuilder(); // if we are a mixed type, don't call ourself dictionary encoded for now so we don't end up doing the wrong thing diff --git a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java index e80ca2595a3..23e86ce45a2 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/StringUtf8ColumnIndexSupplier.java @@ -24,7 +24,6 @@ import org.apache.druid.collections.bitmap.BitmapFactory; import org.apache.druid.collections.bitmap.ImmutableBitmap; import org.apache.druid.collections.spatial.ImmutableRTree; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.StringEncodingStrategies; import org.apache.druid.segment.data.GenericIndexed; @@ -59,34 +58,17 @@ public class StringUtf8ColumnIndexSupplier> @Nullable private final ImmutableRTree indexedTree; - private final ColumnConfig columnConfig; - private final int numRows; - - public StringUtf8ColumnIndexSupplier( - BitmapFactory bitmapFactory, - Supplier utf8Dictionary, - @Nullable GenericIndexed bitmaps, - @Nullable ImmutableRTree indexedTree - ) - { - this(bitmapFactory, utf8Dictionary, bitmaps, indexedTree, ColumnConfig.ALWAYS_USE_INDEXES, Integer.MAX_VALUE); - } - public StringUtf8ColumnIndexSupplier( BitmapFactory bitmapFactory, Supplier utf8Dictionary, @Nullable GenericIndexed bitmaps, - @Nullable ImmutableRTree indexedTree, - @Nullable ColumnConfig columnConfig, - int numRows + @Nullable ImmutableRTree indexedTree ) { this.bitmapFactory = bitmapFactory; this.bitmaps = bitmaps; this.utf8Dictionary = utf8Dictionary; this.indexedTree = indexedTree; - this.columnConfig = columnConfig; - this.numRows = numRows; } @Nullable @@ -129,18 +111,14 @@ public class StringUtf8ColumnIndexSupplier> return (T) new IndexedStringDruidPredicateIndexes<>( bitmapFactory, new StringEncodingStrategies.Utf8ToStringIndexed(dict), - singleThreadedBitmaps, - columnConfig, - numRows + singleThreadedBitmaps ); } else if (clazz.equals(LexicographicalRangeIndexes.class)) { return (T) new IndexedUtf8LexicographicalRangeIndexes<>( bitmapFactory, dict, singleThreadedBitmaps, - dict.get(0) == null, - columnConfig, - numRows + dict.get(0) == null ); } else if ( clazz.equals(DictionaryEncodedStringValueIndex.class) || diff --git a/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java b/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java index c6da18edb8b..9dbe7938464 100644 --- a/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java +++ b/processing/src/main/java/org/apache/druid/segment/vector/FilteredVectorOffset.java @@ -21,7 +21,6 @@ package org.apache.druid.segment.vector; import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.ISE; -import org.apache.druid.query.filter.Filter; import org.apache.druid.query.filter.vector.ReadableVectorMatch; import org.apache.druid.query.filter.vector.VectorMatch; import org.apache.druid.query.filter.vector.VectorValueMatcher; @@ -44,15 +43,12 @@ public class FilteredVectorOffset implements VectorOffset public static FilteredVectorOffset create( final VectorOffset baseOffset, - final VectorColumnSelectorFactory baseColumnSelectorFactory, - final Filter filter + final VectorValueMatcher filterMatcher ) { // This is not the same logic as the row-by-row FilteredOffset, which uses bitmaps whenever possible. // I am not convinced that approach is best in all cases (it's potentially too eager) and also have not implemented // it for vector matchers yet. So let's keep this method simple for now, and try to harmonize them in the future. - Preconditions.checkState(filter.canVectorizeMatcher(baseColumnSelectorFactory), "Cannot vectorize"); - final VectorValueMatcher filterMatcher = filter.makeVectorMatcher(baseColumnSelectorFactory); return new FilteredVectorOffset(baseOffset, filterMatcher); } diff --git a/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java b/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java index f0018b94f7c..11a506cdafa 100644 --- a/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java +++ b/processing/src/test/java/org/apache/druid/query/filter/LikeDimFilterTest.java @@ -98,6 +98,7 @@ public class LikeDimFilterTest extends InitializedNullHandlingTest EqualsVerifier.forClass(LikeDimFilter.LikeMatcher.class) .usingGetClass() .withNonnullFields("suffixMatch", "prefix", "pattern") + .withIgnoredFields("likePattern") .verify(); } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java index 6d6dbef3ab8..9dbd866654c 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexBuilder.java @@ -101,7 +101,7 @@ public class IndexBuilder public static IndexBuilder create() { - return new IndexBuilder(TestHelper.JSON_MAPPER, ColumnConfig.ALWAYS_USE_INDEXES); + return new IndexBuilder(TestHelper.JSON_MAPPER, ColumnConfig.DEFAULT); } public static IndexBuilder create(ColumnConfig columnConfig) @@ -111,7 +111,7 @@ public class IndexBuilder public static IndexBuilder create(ObjectMapper jsonMapper) { - return new IndexBuilder(jsonMapper, ColumnConfig.ALWAYS_USE_INDEXES); + return new IndexBuilder(jsonMapper, ColumnConfig.DEFAULT); } public static IndexBuilder create(ObjectMapper jsonMapper, ColumnConfig columnConfig) diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java index a8d114af9d1..2e328449075 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java @@ -118,7 +118,7 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling @Test public void testV9LoaderThatIgnoresmptyColumns() throws IOException { - QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(ColumnConfig.ALWAYS_USE_INDEXES).load( + QueryableIndex queryableIndex = new V9IndexLoaderExceptEmptyColumns(ColumnConfig.SELECTION_SIZE).load( segmentDir, TestHelper.makeJsonMapper(), false, diff --git a/processing/src/test/java/org/apache/druid/segment/TestHelper.java b/processing/src/test/java/org/apache/druid/segment/TestHelper.java index 8770d3f07fd..8e0728730be 100644 --- a/processing/src/test/java/org/apache/druid/segment/TestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/TestHelper.java @@ -67,9 +67,14 @@ public class TestHelper return new IndexMergerV9(JSON_MAPPER, getTestIndexIO(), segmentWriteOutMediumFactory, true); } + public static IndexMergerV9 getTestIndexMergerV9(SegmentWriteOutMediumFactory segmentWriteOutMediumFactory, ColumnConfig columnConfig) + { + return new IndexMergerV9(JSON_MAPPER, getTestIndexIO(columnConfig), segmentWriteOutMediumFactory, true); + } + public static IndexIO getTestIndexIO() { - return getTestIndexIO(ColumnConfig.ALWAYS_USE_INDEXES); + return getTestIndexIO(ColumnConfig.SELECTION_SIZE); } public static IndexIO getTestIndexIO(ColumnConfig columnConfig) diff --git a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java index 665751aee39..f95bd2ea8e9 100644 --- a/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java +++ b/processing/src/test/java/org/apache/druid/segment/filter/FilterPartitionTest.java @@ -37,13 +37,9 @@ import org.apache.druid.query.filter.DruidObjectPredicate; import org.apache.druid.query.filter.DruidPredicateFactory; import org.apache.druid.query.filter.DruidPredicateMatch; import org.apache.druid.query.filter.Filter; -import org.apache.druid.query.filter.FilterTuning; import org.apache.druid.query.filter.OrDimFilter; import org.apache.druid.query.filter.SelectorDimFilter; -import org.apache.druid.segment.ColumnSelectorColumnIndexSelector; -import org.apache.druid.segment.FilterAnalysis; import org.apache.druid.segment.IndexBuilder; -import org.apache.druid.segment.QueryableIndexStorageAdapter; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException; import org.apache.druid.segment.index.BitmapColumnIndex; @@ -72,15 +68,6 @@ public class FilterPartitionTest extends BaseFilterTest super(dimension, value); } - public NoBitmapSelectorFilter( - String dimension, - String value, - FilterTuning filterTuning - ) - { - super(dimension, value, filterTuning); - } - @Nullable @Override public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector) @@ -735,86 +722,4 @@ public class FilterPartitionTest extends BaseFilterTest ImmutableList.of("2", "3", "4", "6", "7", "9") ); } - - @Test - public void testAnalyze() - { - if (!(adapter instanceof QueryableIndexStorageAdapter)) { - return; - } - QueryableIndexStorageAdapter storageAdapter = (QueryableIndexStorageAdapter) adapter; - final int numRows = adapter.getNumRows(); - - final ColumnSelectorColumnIndexSelector bitmapIndexSelector = storageAdapter.makeBitmapIndexSelector(BaseFilterTest.VIRTUAL_COLUMNS); - - // has bitmap index, will use it by default - Filter normalFilter = new SelectorFilter("dim1", "HELLO"); - FilterAnalysis filterAnalysisNormal = - FilterAnalysis.analyzeFilter(normalFilter, bitmapIndexSelector, null, numRows); - Assert.assertTrue(filterAnalysisNormal.getPreFilterBitmap() != null); - Assert.assertTrue(filterAnalysisNormal.getPostFilter() == null); - - - // no bitmap index, should be a post filter - Filter noBitmapFilter = new NoBitmapSelectorFilter("dim1", "HELLO"); - FilterAnalysis noBitmapFilterAnalysis = - FilterAnalysis.analyzeFilter(noBitmapFilter, bitmapIndexSelector, null, numRows); - Assert.assertTrue(noBitmapFilterAnalysis.getPreFilterBitmap() == null); - Assert.assertTrue(noBitmapFilterAnalysis.getPostFilter() != null); - - // this column has a bitmap index, but is forced to not use it - Filter bitmapFilterWithForceNoIndexTuning = new SelectorFilter( - "dim1", - "HELLO", - new FilterTuning(false, null, null) - ); - FilterAnalysis bitmapFilterWithForceNoIndexTuningAnalysis = - FilterAnalysis.analyzeFilter(bitmapFilterWithForceNoIndexTuning, bitmapIndexSelector, null, numRows); - Assert.assertTrue(bitmapFilterWithForceNoIndexTuningAnalysis.getPreFilterBitmap() == null); - Assert.assertTrue(bitmapFilterWithForceNoIndexTuningAnalysis.getPostFilter() != null); - - // this max cardinality is too low to use bitmap index - Filter bitmapFilterWithCardinalityMax = new SelectorFilter( - "dim1", - "HELLO", - new FilterTuning(true, 0, 3) - ); - FilterAnalysis bitmapFilterWithCardinalityMaxAnalysis = - FilterAnalysis.analyzeFilter(bitmapFilterWithCardinalityMax, bitmapIndexSelector, null, numRows); - Assert.assertTrue(bitmapFilterWithCardinalityMaxAnalysis.getPreFilterBitmap() == null); - Assert.assertTrue(bitmapFilterWithCardinalityMaxAnalysis.getPostFilter() != null); - - // this max cardinality is high enough that we can still use bitmap index - Filter bitmapFilterWithCardinalityMax2 = new SelectorFilter( - "dim1", - "HELLO", - new FilterTuning(true, 0, 1000) - ); - FilterAnalysis bitmapFilterWithCardinalityMax2Analysis = - FilterAnalysis.analyzeFilter(bitmapFilterWithCardinalityMax2, bitmapIndexSelector, null, numRows); - Assert.assertTrue(bitmapFilterWithCardinalityMax2Analysis.getPreFilterBitmap() != null); - Assert.assertTrue(bitmapFilterWithCardinalityMax2Analysis.getPostFilter() == null); - - // this min cardinality is too high, will not use bitmap index - Filter bitmapFilterWithCardinalityMin = new SelectorFilter( - "dim1", - "HELLO", - new FilterTuning(true, 1000, null) - ); - FilterAnalysis bitmapFilterWithCardinalityMinAnalysis = - FilterAnalysis.analyzeFilter(bitmapFilterWithCardinalityMin, bitmapIndexSelector, null, numRows); - Assert.assertTrue(bitmapFilterWithCardinalityMinAnalysis.getPreFilterBitmap() == null); - Assert.assertTrue(bitmapFilterWithCardinalityMinAnalysis.getPostFilter() != null); - - // cannot force using bitmap if there are no bitmaps - Filter noBitmapFilterWithForceUse = new NoBitmapSelectorFilter( - "dim1", - "HELLO", - new FilterTuning(true, null, null) - ); - FilterAnalysis noBitmapFilterWithForceUseAnalysis = - FilterAnalysis.analyzeFilter(noBitmapFilterWithForceUse, bitmapIndexSelector, null, numRows); - Assert.assertTrue(noBitmapFilterWithForceUseAnalysis.getPreFilterBitmap() == null); - Assert.assertTrue(noBitmapFilterWithForceUseAnalysis.getPostFilter() != null); - } } diff --git a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java index 7f44d8db746..3c7ff2389c6 100644 --- a/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java +++ b/processing/src/test/java/org/apache/druid/segment/generator/SegmentGenerator.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.IndexBuilder; import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.TestHelper; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.incremental.IncrementalIndex; import org.apache.druid.segment.incremental.IncrementalIndexSchema; import org.apache.druid.segment.serde.ComplexMetrics; @@ -157,7 +158,7 @@ public class SegmentGenerator implements Closeable if (outDir.exists()) { try { log.info("Found segment with hash[%s] cached in directory[%s].", dataHash, outDir); - return TestHelper.getTestIndexIO().loadIndex(outDir); + return TestHelper.getTestIndexIO(ColumnConfig.DEFAULT).loadIndex(outDir); } catch (IOException e) { throw new RuntimeException(e); @@ -220,7 +221,7 @@ public class SegmentGenerator implements Closeable } else { try { retVal = TestHelper - .getTestIndexIO() + .getTestIndexIO(ColumnConfig.DEFAULT) .loadIndex( TestHelper.getTestIndexMergerV9(OffHeapMemorySegmentWriteOutMediumFactory.instance()) .mergeQueryableIndex( diff --git a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java index b80f4a1e2ec..0746c104732 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java +++ b/processing/src/test/java/org/apache/druid/segment/join/JoinTestHelper.java @@ -199,7 +199,7 @@ public class JoinTestHelper public static IndexBuilder createFactIndexBuilder(final File tmpDir) throws IOException { - return createFactIndexBuilder(ColumnConfig.ALWAYS_USE_INDEXES, tmpDir, -1); + return createFactIndexBuilder(ColumnConfig.SELECTION_SIZE, tmpDir, -1); } public static IndexBuilder createFactIndexBuilder( diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index a97fd378635..cd85768e5ac 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -98,22 +98,6 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest private static final String NO_MATCH = "no"; - private static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() - { - - @Override - public double skipValueRangeIndexScale() - { - return 1.0; - } - - @Override - public double skipValuePredicateIndexScale() - { - return 1.0; - } - }; - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @@ -260,7 +244,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest ); bob.setFileMapper(fileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(baseBuffer, bob, ALWAYS_USE_INDEXES); + deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); @@ -284,7 +268,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest ); bob.setFileMapper(arrayFileMapper); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(arrayBaseBuffer, bob, ALWAYS_USE_INDEXES); + deserializer.read(arrayBaseBuffer, bob, ColumnConfig.SELECTION_SIZE); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); @@ -305,7 +289,7 @@ public class NestedDataColumnSupplierTest extends InitializedNullHandlingTest false, baseBuffer, bob, - ALWAYS_USE_INDEXES, + ColumnConfig.SELECTION_SIZE, bitmapSerdeFactory, ByteOrder.nativeOrder() ); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java index 7b3a8eac20b..bcc40b333c4 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierV4Test.java @@ -51,6 +51,7 @@ import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.column.ColumnBuilder; import org.apache.druid.segment.column.ColumnCapabilities; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnIndexSupplier; import org.apache.druid.segment.column.ColumnType; @@ -223,7 +224,7 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest bob.setFileMapper(fileMapper); ComplexColumnPartSerde partSerde = ComplexColumnPartSerde.createDeserializer(NestedDataComplexTypeSerde.TYPE_NAME); ColumnPartSerde.Deserializer deserializer = partSerde.getDeserializer(); - deserializer.read(baseBuffer, bob, NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES); + deserializer.read(baseBuffer, bob, ColumnConfig.SELECTION_SIZE); final ColumnHolder holder = bob.build(); final ColumnCapabilities capabilities = holder.getCapabilities(); Assert.assertEquals(ColumnType.NESTED_DATA, capabilities.toColumnType()); @@ -245,7 +246,7 @@ public class NestedDataColumnSupplierV4Test extends InitializedNullHandlingTest NestedDataColumnSupplierV4 supplier = NestedDataColumnSupplierV4.read( baseBuffer, bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES, + ColumnConfig.SELECTION_SIZE, NestedDataComplexTypeSerde.OBJECT_MAPPER ); final String expectedReason = "none"; diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java index 8ed9ad23fdb..c9d7e05c622 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldColumnIndexSupplierTest.java @@ -66,21 +66,6 @@ import java.util.TreeSet; public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingTest { - private static final int ROW_COUNT = 10; - static final ColumnConfig ALWAYS_USE_INDEXES = new ColumnConfig() - { - @Override - public double skipValueRangeIndexScale() - { - return 1.0; - } - - @Override - public double skipValuePredicateIndexScale() - { - return 1.0; - } - }; BitmapSerdeFactory roaringFactory = RoaringBitmapSerdeFactory.getInstance(); BitmapResultFactory bitmapResultFactory = new DefaultBitmapResultFactory( roaringFactory.getBitmapFactory() @@ -1317,7 +1302,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT .getByteValue() ), roaringFactory.getBitmapFactory(), - ALWAYS_USE_INDEXES, + ColumnConfig.SELECTION_SIZE, bitmaps, dictionarySupplier, stringIndexed, @@ -1325,8 +1310,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT doubleIndexed, globalArrays, null, - null, - ROW_COUNT + null ); StringValueSetIndexes valueSetIndex = indexSupplier.as(StringValueSetIndexes.class); @@ -1352,107 +1336,9 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT checkBitmap(bitmap); } - @Test - public void testSkipIndexThresholds() throws IOException - { - ColumnConfig twentyPercent = new ColumnConfig() - { - @Override - public double skipValueRangeIndexScale() - { - return 0.2; - } - - @Override - public double skipValuePredicateIndexScale() - { - return 0.2; - } - }; - NestedFieldColumnIndexSupplier singleTypeStringSupplier = makeSingleTypeStringSupplier(twentyPercent); - NestedFieldColumnIndexSupplier singleTypeLongSupplier = makeSingleTypeLongSupplier(twentyPercent); - NestedFieldColumnIndexSupplier singleTypeDoubleSupplier = makeSingleTypeDoubleSupplier(twentyPercent); - NestedFieldColumnIndexSupplier variantSupplierWithNull = makeVariantSupplierWithNull(twentyPercent); - - // value cardinality of all of these dictionaries is bigger than the skip threshold, so predicate index short - // circuit early and return nothing - DruidPredicateFactory predicateFactory = new InDimFilter.InFilterDruidPredicateFactory( - null, - InDimFilter.ValuesSet.copyOf(ImmutableSet.of("0")) - ); - Assert.assertNull(singleTypeStringSupplier.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); - Assert.assertNull(singleTypeLongSupplier.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); - Assert.assertNull(singleTypeDoubleSupplier.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); - Assert.assertNull(variantSupplierWithNull.as(DruidPredicateIndexes.class).forPredicate(predicateFactory)); - - // range index computation is a bit more complicated and done inside of the index maker gizmo because we don't know - // the range up front - LexicographicalRangeIndexes stringRange = singleTypeStringSupplier.as(LexicographicalRangeIndexes.class); - NumericRangeIndexes longRanges = singleTypeLongSupplier.as(NumericRangeIndexes.class); - NumericRangeIndexes doubleRanges = singleTypeDoubleSupplier.as(NumericRangeIndexes.class); - - // string: [b, foo, fooo, z] - // small enough should be cool - Assert.assertNotNull(stringRange.forRange("fo", false, "fooo", false)); - Assert.assertNotNull(stringRange.forRange("fo", false, "fooo", false, DruidObjectPredicate.alwaysTrue())); - // range too big, no index - Assert.assertNull(stringRange.forRange("fo", false, "z", false)); - Assert.assertNull(stringRange.forRange("fo", false, "z", false, DruidObjectPredicate.alwaysTrue())); - - // long: [1, 3, 100, 300] - // small enough should be cool - Assert.assertNotNull(longRanges.forRange(1, false, 100, true)); - // range too big, no index - Assert.assertNull(longRanges.forRange(1, false, null, false)); - - // double: [1.1, 1.2, 3.3, 6.6] - // small enough should be cool - Assert.assertNotNull(doubleRanges.forRange(null, false, 1.2, false)); - // range too big, no index - Assert.assertNull(doubleRanges.forRange(null, false, 3.3, false)); - - // other index types should not be impacted - Assert.assertNotNull(singleTypeStringSupplier.as(DictionaryEncodedStringValueIndex.class)); - Assert.assertNotNull(singleTypeStringSupplier.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(singleTypeStringSupplier.as(StringValueSetIndexes.class).forValue("foo")); - Assert.assertNotNull( - singleTypeStringSupplier.as(StringValueSetIndexes.class) - .forSortedValues(new TreeSet<>(ImmutableSet.of("foo", "fooo", "z"))) - ); - Assert.assertNotNull(singleTypeStringSupplier.as(NullValueIndex.class)); - - Assert.assertNotNull(singleTypeLongSupplier.as(DictionaryEncodedStringValueIndex.class)); - Assert.assertNotNull(singleTypeLongSupplier.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(singleTypeLongSupplier.as(StringValueSetIndexes.class).forValue("1")); - Assert.assertNotNull( - singleTypeLongSupplier.as(StringValueSetIndexes.class) - .forSortedValues(new TreeSet<>(ImmutableSet.of("1", "3", "100"))) - ); - Assert.assertNotNull(singleTypeLongSupplier.as(NullValueIndex.class)); - - Assert.assertNotNull(singleTypeDoubleSupplier.as(DictionaryEncodedStringValueIndex.class)); - Assert.assertNotNull(singleTypeDoubleSupplier.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(singleTypeDoubleSupplier.as(StringValueSetIndexes.class).forValue("1.1")); - Assert.assertNotNull( - singleTypeDoubleSupplier.as(StringValueSetIndexes.class) - .forSortedValues(new TreeSet<>(ImmutableSet.of("1.1", "1.2", "3.3"))) - ); - Assert.assertNotNull(singleTypeDoubleSupplier.as(NullValueIndex.class)); - - // variant: [null, b, z, 1, 300, 1.1, 9.9] - Assert.assertNotNull(variantSupplierWithNull.as(DictionaryEncodedStringValueIndex.class)); - Assert.assertNotNull(variantSupplierWithNull.as(DictionaryEncodedValueIndex.class)); - Assert.assertNotNull(variantSupplierWithNull.as(StringValueSetIndexes.class).forValue("b")); - Assert.assertNotNull( - variantSupplierWithNull.as(StringValueSetIndexes.class) - .forSortedValues(new TreeSet<>(ImmutableSet.of("b", "1", "9.9"))) - ); - Assert.assertNotNull(variantSupplierWithNull.as(NullValueIndex.class)); - } - private NestedFieldColumnIndexSupplier makeSingleTypeStringSupplier() throws IOException { - return makeSingleTypeStringSupplier(ALWAYS_USE_INDEXES); + return makeSingleTypeStringSupplier(ColumnConfig.SELECTION_SIZE); } private NestedFieldColumnIndexSupplier makeSingleTypeStringSupplier(ColumnConfig columnConfig) throws IOException @@ -1526,14 +1412,13 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT globalDoubles, globalArrays, null, - null, - ROW_COUNT + null ); } private NestedFieldColumnIndexSupplier makeSingleTypeStringWithNullsSupplier() throws IOException { - return makeSingleTypeStringWithNullsSupplier(ALWAYS_USE_INDEXES); + return makeSingleTypeStringWithNullsSupplier(ColumnConfig.SELECTION_SIZE); } private NestedFieldColumnIndexSupplier makeSingleTypeStringWithNullsSupplier(ColumnConfig columnConfig) @@ -1611,14 +1496,13 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT globalDoubles, globalArrays, null, - null, - ROW_COUNT + null ); } private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplier() throws IOException { - return makeSingleTypeLongSupplier(ALWAYS_USE_INDEXES); + return makeSingleTypeLongSupplier(ColumnConfig.SELECTION_SIZE); } private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplier(ColumnConfig columnConfig) throws IOException @@ -1692,14 +1576,13 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT globalDoubles, globalArrays, null, - null, - ROW_COUNT + null ); } private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplierWithNull() throws IOException { - return makeSingleTypeLongSupplierWithNull(ALWAYS_USE_INDEXES); + return makeSingleTypeLongSupplierWithNull(ColumnConfig.SELECTION_SIZE); } private NestedFieldColumnIndexSupplier makeSingleTypeLongSupplierWithNull(ColumnConfig columnConfig) @@ -1778,14 +1661,13 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT globalDoubles, globalArrays, null, - null, - ROW_COUNT + null ); } private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplier() throws IOException { - return makeSingleTypeDoubleSupplier(ALWAYS_USE_INDEXES); + return makeSingleTypeDoubleSupplier(ColumnConfig.SELECTION_SIZE); } private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplier(ColumnConfig columnConfig) throws IOException @@ -1859,14 +1741,13 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT globalDoubles, globalArrays, null, - null, - ROW_COUNT + null ); } private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplierWithNull() throws IOException { - return makeSingleTypeDoubleSupplierWithNull(ALWAYS_USE_INDEXES); + return makeSingleTypeDoubleSupplierWithNull(ColumnConfig.SELECTION_SIZE); } private NestedFieldColumnIndexSupplier makeSingleTypeDoubleSupplierWithNull(ColumnConfig columnConfig) @@ -1945,14 +1826,13 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT globalDoubles, globalArrays, null, - null, - ROW_COUNT + null ); } private NestedFieldColumnIndexSupplier makeVariantSupplierWithNull() throws IOException { - return makeVariantSupplierWithNull(ALWAYS_USE_INDEXES); + return makeVariantSupplierWithNull(ColumnConfig.SELECTION_SIZE); } private NestedFieldColumnIndexSupplier makeVariantSupplierWithNull(ColumnConfig columnConfig) throws IOException @@ -2041,8 +1921,7 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT globalDoubles, globalArrays, null, - null, - ROW_COUNT + null ); } diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java index 2e2a8a21e29..5d76e18f94c 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarDoubleColumnSupplierTest.java @@ -41,6 +41,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexableAdapter; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; @@ -191,7 +192,7 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + ColumnConfig.SELECTION_SIZE ); try (ScalarDoubleColumn column = (ScalarDoubleColumn) supplier.get()) { smokeTest(supplier, column); @@ -209,7 +210,7 @@ public class ScalarDoubleColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + ColumnConfig.SELECTION_SIZE ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java index 801b88c3beb..4661d72cdbb 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarLongColumnSupplierTest.java @@ -41,6 +41,7 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexableAdapter; import org.apache.druid.segment.SimpleAscendingOffset; import org.apache.druid.segment.column.ColumnBuilder; +import org.apache.druid.segment.column.ColumnConfig; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.data.BitmapSerdeFactory; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; @@ -191,7 +192,7 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + ColumnConfig.SELECTION_SIZE ); try (ScalarLongColumn column = (ScalarLongColumn) supplier.get()) { smokeTest(supplier, column); @@ -209,7 +210,7 @@ public class ScalarLongColumnSupplierTest extends InitializedNullHandlingTest bitmapSerdeFactory, baseBuffer, bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + ColumnConfig.SELECTION_SIZE ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java index cc9edb6a357..5f0c1d020c8 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/ScalarStringColumnSupplierTest.java @@ -191,8 +191,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + bob ); try (StringUtf8DictionaryEncodedColumn column = (StringUtf8DictionaryEncodedColumn) supplier.get()) { smokeTest(supplier, column); @@ -209,8 +208,7 @@ public class ScalarStringColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + bob ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java index 4604aff1c20..6ea4509bef7 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/VariantColumnSupplierTest.java @@ -317,8 +317,7 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + bob ); try (VariantColumn column = (VariantColumn) supplier.get()) { smokeTest(supplier, column, data, expectedTypes); @@ -336,8 +335,7 @@ public class VariantColumnSupplierTest extends InitializedNullHandlingTest ByteOrder.nativeOrder(), bitmapSerdeFactory, baseBuffer, - bob, - NestedFieldColumnIndexSupplierTest.ALWAYS_USE_INDEXES + bob ); final String expectedReason = "none"; final AtomicReference failureReason = new AtomicReference<>(expectedReason);