adaptive filter partitioning (#15838)

* cooler cursor filter processing allowing much smart utilization of indexes by feeding selectivity forward, with implementations for range and predicate based filters
* added new method Filter.makeFilterBundle which cursors use to get indexes and matchers for building offsets
* AND filter partitioning is now pushed all the way down, even to nested AND filters
* vector engine now uses same indexed base value matcher strategy for OR filters which partially support indexes
This commit is contained in:
Clint Wylie 2024-02-29 15:38:12 -08:00 committed by GitHub
parent baaa4a6808
commit 101176590c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
70 changed files with 1808 additions and 1109 deletions

View File

@ -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<DimensionSchema> 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<PlannerFactory, SqlEngine> 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<Object[]> 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<Object[]> resultSequence = plannerResult.run().getResults();
final Yielder<Object[]> 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()

View File

@ -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);
}

View File

@ -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<DimensionSchema> dims = ImmutableList.<DimensionSchema>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<DimensionSchema> 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<DimensionSchema> 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,

View File

@ -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

View File

@ -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);
}

View File

@ -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)
{

View File

@ -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());

View File

@ -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));
}
}

View File

@ -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));
}
}

View File

@ -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)
{

View File

@ -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(

View File

@ -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<Object, ExprEval<?>> evalFunction =
inputValue -> expr.eval(InputBindings.forInputSupplier(inputColumn, inputType, () -> inputValue));
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(inputColumnIndexes.getCardinality())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)

View File

@ -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<QueryType extends Query<?>> implements QueryMet
setDimension("segment", segmentIdentifier);
}
@Override
public void preFilters(List<Filter> preFilters)
{
// Emit nothing by default.
}
@Override
public void postFilters(List<Filter> postFilters)
{
// Emit nothing by default.
}
@Override
public void identity(String identity)
{

View File

@ -177,12 +177,6 @@ public class DruidProcessingConfig implements ColumnConfig
return indexes.getSkipValueRangeIndexScale();
}
@Override
public double skipValuePredicateIndexScale()
{
return indexes.getSkipValuePredicateIndexScale();
}
public boolean isNumThreadsConfigured()
{
return numThreadsConfigured;

View File

@ -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;
}
}

View File

@ -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<QueryType extends Query<?>>
void segment(String segmentIdentifier);
void preFilters(List<Filter> 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<Filter> preFilters)
{
// do nothing, nothing calls this
}
void postFilters(List<Filter> 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<Filter> 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<QueryType extends Query<?>>
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<QueryType extends Query<?>>
QueryMetrics<QueryType> 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<QueryType> reportBitmapConstructionTime(long timeNs);
@ -359,8 +387,8 @@ public interface QueryMetrics<QueryType extends Query<?>>
QueryMetrics<QueryType> 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<QueryType> reportPreFilteredRows(long numRows);

View File

@ -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<Filter> getFilters();
/**
* Get a ValueMatcher that applies this filter to row values.
*
* Unlike makeMatcher(ValueMatcherFactory), this method allows the Filter to utilize bitmap indexes.
*
* An implementation should either:
* - return a ValueMatcher that checks row values, using the provided ValueMatcherFactory
* - or, if possible, get a bitmap index for this filter using the 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<String> getRequiredColumns()
{
@ -68,5 +42,4 @@ public interface BooleanFilter extends Filter
}
return allColumns;
}
}

View File

@ -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}
*/
<T> DimFilterToStringBuilder append(T s)
public <T> DimFilterToStringBuilder append(T s)
{
builder.append(s);
return this;

View File

@ -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)}.
* <p>
* 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 <T> - Type of {@link BitmapResultFactory} results, {@link ImmutableBitmap} by default
*/
default <T> FilterBundle makeFilterBundle(
ColumnIndexSelector columnIndexSelector,
BitmapResultFactory<T> 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

View File

@ -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.
* <p>
* 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<ColumnSelectorFactory, ValueMatcher> matcherFn;
private final Function<VectorColumnSelectorFactory, VectorValueMatcher> vectorMatcherFn;
public SimpleMatcherBundle(
MatcherBundleInfo matcherInfo,
Function<ColumnSelectorFactory, ValueMatcher> matcherFn,
Function<VectorColumnSelectorFactory, VectorValueMatcher> 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<String> filter;
private final List<IndexBundleInfo> indexes;
private final int selectionSize;
private final long buildTimeNs;
public IndexBundleInfo(
Supplier<String> filterString,
int selectionSize,
long buildTimeNs,
@Nullable List<IndexBundleInfo> 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<IndexBundleInfo> 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<String> filter;
@Nullable
final List<MatcherBundleInfo> matchers;
@Nullable
private final IndexBundleInfo partialIndex;
public MatcherBundleInfo(
Supplier<String> filter,
@Nullable IndexBundleInfo partialIndex,
@Nullable List<MatcherBundleInfo> 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<MatcherBundleInfo> getMatchers()
{
return matchers;
}
@Override
public String toString()
{
return "{" +
"filter=\"" + filter.get() + '\"' +
(partialIndex != null ? ", partialIndex=" + partialIndex : "") +
(matchers != null ? ", matchers=" + matchers : "") +
'}';
}
}
}

View File

@ -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

View File

@ -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;
}
}
}

View File

@ -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);
}

View File

@ -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<Filter> preFilters)
public void filterBundle(FilterBundle.BundleInfo bundleInfo)
{
delegateQueryMetrics.preFilters(preFilters);
}
@Override
public void postFilters(List<Filter> postFilters)
{
delegateQueryMetrics.postFilters(postFilters);
delegateQueryMetrics.filterBundle(bundleInfo);
}
@Override

View File

@ -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<TopNQuery>
/**
* 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);

View File

@ -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)) {

View File

@ -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<Filter> preFilters;
final List<Filter> 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;
}
}

View File

@ -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);
}
};
}
}
}
}

View File

@ -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<? extends Query> 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

View File

@ -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)
);
}
}

View File

@ -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.
* <p>
* 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.
* <p>
* Currently only the {@link org.apache.druid.segment.nested.NestedCommonFormatColumn} implementations of
* {@link ColumnIndexSupplier} support this behavior.
* <p>
* 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}.
* <p>
* 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.
* <p>
* 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;
}
}

View File

@ -44,14 +44,4 @@ public interface ColumnIndexSupplier
*/
@Nullable
<T> T as(Class<T> 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);
}
}

View File

@ -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<Filter> filters;
@ -73,21 +72,136 @@ public class AndFilter implements BooleanFilter
this(new LinkedHashSet<>(filters));
}
public static <T> ImmutableBitmap getBitmapIndex(
ColumnIndexSelector selector,
@Override
public <T> FilterBundle makeFilterBundle(
ColumnIndexSelector columnIndexSelector,
BitmapResultFactory<T> bitmapResultFactory,
List<Filter> filters
int selectionRowCount,
int totalRowCount,
boolean includeUnknown
)
{
return bitmapResultFactory.toImmutableBitmap(
getBitmapIndex(selector, filters).computeBitmapResult(bitmapResultFactory, false)
final List<FilterBundle.IndexBundleInfo> indexBundleInfos = new ArrayList<>();
final List<FilterBundle.MatcherBundle> matcherBundles = new ArrayList<>();
final List<FilterBundle.MatcherBundleInfo> 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<Filter> 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> T computeBitmapResult(
BitmapResultFactory<T> bitmapResultFactory,
int selectionRowCount,
int totalRowCount,
boolean includeUnknown
)
{
final List<T> 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<ValueMatcher> matchers = new ArrayList<>();
final List<ImmutableBitmap> 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<Filter> 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);
}

View File

@ -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();
}

View File

@ -89,6 +89,38 @@ public class IsBooleanFilter implements Filter
selector.getNumRows()
);
}
@Nullable
@Override
public <T> T computeBitmapResult(
BitmapResultFactory<T> 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;

View File

@ -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<String> getRequiredColumns()
{

View File

@ -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();
}
}

View File

@ -88,6 +88,27 @@ public class NotFilter implements Filter
selector.getNumRows()
);
}
@Nullable
@Override
public <T> T computeBitmapResult(
BitmapResultFactory<T> 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;

View File

@ -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 <T> FilterBundle makeFilterBundle(
ColumnIndexSelector columnIndexSelector,
BitmapResultFactory<T> 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<FilterBundle.IndexBundle> indexOnlyBundles = new ArrayList<>();
final List<FilterBundle.IndexBundleInfo> indexOnlyBundlesInfo = new ArrayList<>();
final List<FilterBundle.MatcherBundle> partialIndexBundles = new ArrayList<>();
final List<FilterBundle.MatcherBundle> 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<FilterBundle.MatcherBundle> allMatcherBundles = Lists.newArrayListWithCapacity(estimatedSize);
final List<FilterBundle.MatcherBundleInfo> 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> T computeBitmapResult(
BitmapResultFactory<T> bitmapResultFactory,
int selectionRowCount,
int totalRowCount,
boolean includeUnknown
)
{
List<T> 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<ValueMatcher> matchers = new ArrayList<>();
final List<ImmutableBitmap> 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<Filter> 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<FilterBundle.IndexBundle> indexOnlyBundles,
List<FilterBundle.IndexBundleInfo> 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();
}
};
}
}

View File

@ -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<String> getRequiredColumns()
{

View File

@ -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> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown);
<T> T computeBitmapResult(
BitmapResultFactory<T> 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> T computeBitmapResult(
BitmapResultFactory<T> bitmapResultFactory,
int selectionRowCount,
int totalRowCount,
boolean includeUnknown
)
{
return computeBitmapResult(bitmapResultFactory, includeUnknown);
}
}

View File

@ -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()}.
* <p>
* 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> T computeBitmapResult(
BitmapResultFactory<T> bitmapResultFactory,
int selectionRowCount,
int totalRowCount,
boolean includeUnknown
)
{
final int scale = (int) Math.ceil(sizeScale * selectionRowCount);
if (rangeSize > scale) {
return null;
}
return computeBitmapResult(bitmapResultFactory, includeUnknown);
}
}

View File

@ -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> T computeBitmapResult(
BitmapResultFactory<T> bitmapResultFactory,
int selectionRowCount,
int totalRowCount,
boolean includeUnknown
)
{
if (selectionRowCount != totalRowCount && selectionRowCount < (dictionarySize * scaleThreshold)) {
return null;
}
return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable(includeUnknown));
}
}

View File

@ -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<TDictionary extends Indexe
private final BitmapFactory bitmapFactory;
private final TDictionary dictionary;
private final Indexed<ImmutableBitmap> bitmaps;
private final ColumnConfig columnConfig;
private final int numRows;
public IndexedStringDruidPredicateIndexes(
BitmapFactory bitmapFactory,
TDictionary dictionary,
Indexed<ImmutableBitmap> bitmaps,
@Nullable ColumnConfig columnConfig,
int numRows
Indexed<ImmutableBitmap> 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<String> stringPredicate = matcherFactory.makeStringPredicate();
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(dictionary.size())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)

View File

@ -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<TDictionary extends In
private final Indexed<ImmutableBitmap> bitmaps;
private final boolean hasNull;
private final ColumnConfig columnConfig;
private final int numRows;
public IndexedUtf8LexicographicalRangeIndexes(
BitmapFactory bitmapFactory,
TDictionary dictionary,
Indexed<ImmutableBitmap> 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<TDictionary extends In
this.dictionary = dictionary;
this.bitmaps = bitmaps;
this.hasNull = hasNull;
this.columnConfig = columnConfig;
this.numRows = numRows;
}
@Override
@ -80,10 +71,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
{
final IntIntPair range = getRange(startValue, startStrict, endValue, endStrict);
final int start = range.leftInt(), end = range.rightInt();
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, end - start)) {
return null;
}
return new SimpleImmutableBitmapDelegatingIterableIndex()
return new DictionaryRangeScanningBitmapIndex(1.0, end - start)
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable()
@ -132,10 +120,7 @@ public final class IndexedUtf8LexicographicalRangeIndexes<TDictionary extends In
{
final IntIntPair range = getRange(startValue, startStrict, endValue, endStrict);
final int start = range.leftInt(), end = range.rightInt();
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, end - start)) {
return null;
}
return new SimpleImmutableBitmapDelegatingIterableIndex()
return new DictionaryRangeScanningBitmapIndex(1.0, end - start)
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable()

View File

@ -29,7 +29,7 @@ import java.util.Collections;
public abstract class SimpleImmutableBitmapDelegatingIterableIndex extends SimpleBitmapColumnIndex
{
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
public final <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
if (includeUnknown) {
final ImmutableBitmap unknownsBitmap = getUnknownsBitmap();

View File

@ -27,9 +27,8 @@ import org.apache.druid.query.BitmapResultFactory;
*/
public abstract class SimpleImmutableBitmapIterableIndex extends SimpleBitmapColumnIndex
{
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
public final <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
return bitmapResultFactory.unionDimensionValueBitmaps(getBitmapIterable(includeUnknown));
}

View File

@ -999,10 +999,6 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
.setHasNulls(hasNull)
.setDictionaryEncodedColumnSupplier(columnSupplier);
final int size;
try (ColumnarInts throwAway = ints.get()) {
size = throwAway.size();
}
columnBuilder.setIndexSupplier(
new NestedFieldColumnIndexSupplier(
types,
@ -1015,8 +1011,7 @@ public abstract class CompressedNestedDataComplexColumn<TStringDictionary extend
doubleDictionarySupplier,
arrayDictionarySupplier,
arrayElementDictionarySupplier,
arrayElementBitmaps,
size
arrayElementBitmaps
),
true,
false

View File

@ -63,10 +63,11 @@ import org.apache.druid.segment.data.GenericIndexed;
import org.apache.druid.segment.data.Indexed;
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.ArrayElementIndexes;
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
import org.apache.druid.segment.index.semantic.DictionaryEncodedValueIndex;
@ -113,7 +114,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
private final int adjustDoubleId;
private final int adjustArrayId;
private final ColumnConfig columnConfig;
private final int numRows;
public NestedFieldColumnIndexSupplier(
FieldTypeInfo.TypeSet types,
@ -126,8 +126,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
Supplier<FixedIndexed<Double>> globalDoubleDictionarySupplier,
@Nullable Supplier<FrontCodedIntArrayIndexed> globalArrayDictionarySupplier,
@Nullable Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier,
@Nullable GenericIndexed<ImmutableBitmap> arrayElementBitmaps,
int numRows
@Nullable GenericIndexed<ImmutableBitmap> arrayElementBitmaps
)
{
this.singleType = types.getSingleType();
@ -144,7 +143,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
this.adjustDoubleId = adjustLongId + globalLongDictionarySupplier.get().size();
this.adjustArrayId = adjustDoubleId + globalDoubleDictionarySupplier.get().size();
this.columnConfig = columnConfig;
this.numRows = numRows;
}
@Nullable
@ -303,7 +301,8 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
boolean endStrict,
Indexed<Integer> localDictionary,
Indexed<T> globalDictionary,
int adjust
int adjust,
double rangeScale
)
{
final IntIntPair localRange = getLocalRangeFromDictionary(
@ -318,10 +317,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
final int startIndex = localRange.leftInt();
final int endIndex = localRange.rightInt();
final int size = endIndex - startIndex;
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, size)) {
return null;
}
return new SimpleImmutableBitmapDelegatingIterableIndex()
return new DictionaryRangeScanningBitmapIndex(rangeScale, size)
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable()
@ -509,7 +505,8 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
endStrict,
localDictionarySupplier.get(),
globalStringDictionarySupplier.get(),
0
0,
1.0
);
}
@ -535,10 +532,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
0
);
final int start = range.leftInt(), end = range.rightInt();
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, end - start)) {
return null;
}
return new SimpleImmutableBitmapDelegatingIterableIndex()
return new DictionaryRangeScanningBitmapIndex(1.0, end - start)
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable()
@ -606,10 +600,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
{
final FixedIndexed<Integer> localDictionary = localDictionarySupplier.get();
if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, localDictionary.size())) {
return null;
}
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(localDictionary.size())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
@ -875,7 +866,8 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
endStrict,
localDictionarySupplier.get(),
globalLongDictionarySupplier.get(),
adjustLongId
adjustLongId,
columnConfig.skipValueRangeIndexScale()
);
}
}
@ -887,11 +879,9 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
{
final FixedIndexed<Integer> localDictionary = localDictionarySupplier.get();
if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, localDictionary.size())) {
return null;
}
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(localDictionary.size())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
{
@ -1142,7 +1132,8 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
endStrict,
localDictionarySupplier.get(),
globalDoubleDictionarySupplier.get(),
adjustDoubleId
adjustDoubleId,
columnConfig.skipValueRangeIndexScale()
);
}
}
@ -1154,10 +1145,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
{
final FixedIndexed<Integer> localDictionary = localDictionarySupplier.get();
if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, localDictionary.size())) {
return null;
}
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(localDictionary.size())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)
@ -1379,10 +1367,7 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
@Nullable
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
{
if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, localDictionary.size())) {
return null;
}
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(localDictionary.size())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)

View File

@ -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<NestedCommon
bitmapSerdeFactory.getObjectStrategy(),
columnBuilder.getFileMapper()
);
final int size;
try (ColumnarDoubles throwAway = doubles.get()) {
size = throwAway.size();
}
return new ScalarDoubleColumnAndIndexSupplier(
doubleDictionarySupplier,
doubles,
rBitmaps,
bitmapSerdeFactory.getBitmapFactory(),
columnConfig,
size
columnConfig
);
}
catch (IOException ex) {
@ -158,15 +154,13 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
private final BitmapFactory bitmapFactory;
private final ImmutableBitmap nullValueBitmap;
private final ColumnConfig columnConfig;
private final int numRows;
private ScalarDoubleColumnAndIndexSupplier(
Supplier<FixedIndexed<Double>> longDictionary,
Supplier<ColumnarDoubles> valueColumnSupplier,
GenericIndexed<ImmutableBitmap> valueIndexes,
BitmapFactory bitmapFactory,
ColumnConfig columnConfig,
int numRows
ColumnConfig columnConfig
)
{
this.doubleDictionarySupplier = longDictionary;
@ -175,7 +169,6 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
this.bitmapFactory = bitmapFactory;
this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0);
this.columnConfig = columnConfig;
this.numRows = numRows;
}
@Override
@ -424,10 +417,10 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
final int startIndex = range.leftInt();
final int endIndex = range.rightInt();
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, endIndex - startIndex)) {
return null;
}
return new SimpleImmutableBitmapDelegatingIterableIndex()
return new DictionaryRangeScanningBitmapIndex(
columnConfig.skipValueRangeIndexScale(),
endIndex - startIndex
)
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable()
@ -467,10 +460,7 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
{
final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, dictionary.size())) {
return null;
}
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(dictionary.size())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)

View File

@ -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<NestedCommonFo
longsValueColumn,
byteOrder
);
final int size;
try (ColumnarLongs throwAway = longs.get()) {
size = throwAway.size();
}
return new ScalarLongColumnAndIndexSupplier(
longDictionarySupplier,
longs,
rBitmaps,
bitmapSerdeFactory.getBitmapFactory(),
columnConfig,
size
columnConfig
);
}
catch (IOException ex) {
@ -158,15 +154,13 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
private final ImmutableBitmap nullValueBitmap;
private final ColumnConfig columnConfig;
private final int numRows;
private ScalarLongColumnAndIndexSupplier(
Supplier<FixedIndexed<Long>> longDictionarySupplier,
Supplier<ColumnarLongs> valueColumnSupplier,
GenericIndexed<ImmutableBitmap> valueIndexes,
BitmapFactory bitmapFactory,
ColumnConfig columnConfig,
int numRows
ColumnConfig columnConfig
)
{
this.longDictionarySupplier = longDictionarySupplier;
@ -175,7 +169,6 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
this.bitmapFactory = bitmapFactory;
this.nullValueBitmap = valueIndexes.get(0) == null ? bitmapFactory.makeEmptyImmutableBitmap() : valueIndexes.get(0);
this.columnConfig = columnConfig;
this.numRows = numRows;
}
@Override
@ -434,10 +427,10 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
final int startIndex = range.leftInt();
final int endIndex = range.rightInt();
if (ColumnIndexSupplier.skipComputingRangeIndexes(columnConfig, numRows, endIndex - startIndex)) {
return null;
}
return new SimpleImmutableBitmapDelegatingIterableIndex()
return new DictionaryRangeScanningBitmapIndex(
columnConfig.skipValueRangeIndexScale(),
endIndex - startIndex
)
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable()
@ -477,10 +470,7 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
public BitmapColumnIndex forPredicate(DruidPredicateFactory matcherFactory)
{
FixedIndexed<Long> dictionary = longDictionarySupplier.get();
if (ColumnIndexSupplier.skipComputingPredicateIndexes(columnConfig, numRows, dictionary.size())) {
return null;
}
return new SimpleImmutableBitmapIterableIndex()
return new DictionaryScanningBitmapIndex(dictionary.size())
{
@Override
public Iterable<ImmutableBitmap> getBitmapIterable(boolean includeUnknown)

View File

@ -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<NestedCommon
ByteOrder byteOrder,
BitmapSerdeFactory bitmapSerdeFactory,
ByteBuffer bb,
ColumnBuilder columnBuilder,
ColumnConfig columnConfig
ColumnBuilder columnBuilder
)
{
final byte version = bb.get();
@ -92,17 +90,11 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
bitmapSerdeFactory.getObjectStrategy(),
columnBuilder.getFileMapper()
);
final int size;
try (ColumnarInts throwAway = ints.get()) {
size = throwAway.size();
}
return new ScalarStringColumnAndIndexSupplier(
dictionarySupplier,
ints,
valueIndexes,
bitmapSerdeFactory,
columnConfig,
size
bitmapSerdeFactory
);
}
catch (IOException ex) {
@ -122,9 +114,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
Supplier<? extends Indexed<ByteBuffer>> dictionarySupplier,
Supplier<ColumnarInts> encodedColumnSupplier,
GenericIndexed<ImmutableBitmap> valueIndexes,
BitmapSerdeFactory serdeFactory,
ColumnConfig columnConfig,
int numRows
BitmapSerdeFactory serdeFactory
)
{
this.dictionarySupplier = dictionarySupplier;
@ -134,9 +124,7 @@ public class ScalarStringColumnAndIndexSupplier implements Supplier<NestedCommon
serdeFactory.getBitmapFactory(),
dictionarySupplier,
valueIndexes,
null,
columnConfig,
numRows
null
);
}

View File

@ -32,7 +32,6 @@ import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.query.BitmapResultFactory;
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.ColumnType;
import org.apache.druid.segment.column.StringEncodingStrategies;
@ -68,8 +67,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
ByteOrder byteOrder,
BitmapSerdeFactory bitmapSerdeFactory,
ByteBuffer bb,
ColumnBuilder columnBuilder,
ColumnConfig columnConfig
ColumnBuilder columnBuilder
)
{
final byte version = bb.get();
@ -174,10 +172,6 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
arrayDictionarybuffer,
byteOrder
);
final int size;
try (ColumnarInts throwAway = ints.get()) {
size = throwAway.size();
}
arrayElementDictionarySupplier = FixedIndexed.read(
arrayElementDictionaryBuffer,
CompressedNestedDataComplexColumn.INT_TYPE_STRATEGY,
@ -195,9 +189,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
ints,
valueIndexes,
arrayElementIndexes,
bitmapSerdeFactory.getBitmapFactory(),
columnConfig,
size
bitmapSerdeFactory.getBitmapFactory()
);
}
catch (IOException ex) {
@ -219,9 +211,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
private final Supplier<FrontCodedIntArrayIndexed> arrayDictionarySupplier;
private final Supplier<FixedIndexed<Integer>> arrayElementDictionarySupplier;
private final Supplier<ColumnarInts> encodedValueColumnSupplier;
@SuppressWarnings("unused")
private final GenericIndexed<ImmutableBitmap> valueIndexes;
@SuppressWarnings("unused")
private final GenericIndexed<ImmutableBitmap> arrayElementIndexes;
private final ImmutableBitmap nullValueBitmap;
@ -236,9 +226,7 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
Supplier<ColumnarInts> encodedValueColumnSupplier,
GenericIndexed<ImmutableBitmap> valueIndexes,
GenericIndexed<ImmutableBitmap> elementIndexes,
BitmapFactory bitmapFactory,
@SuppressWarnings("unused") ColumnConfig columnConfig,
@SuppressWarnings("unused") int numRows
BitmapFactory bitmapFactory
)
{
this.logicalType = logicalType;

View File

@ -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

View File

@ -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<TIndexed extends Indexed<ByteBuffer>>
@Nullable
private final ImmutableRTree indexedTree;
private final ColumnConfig columnConfig;
private final int numRows;
public StringUtf8ColumnIndexSupplier(
BitmapFactory bitmapFactory,
Supplier<TIndexed> utf8Dictionary,
@Nullable GenericIndexed<ImmutableBitmap> bitmaps,
@Nullable ImmutableRTree indexedTree
)
{
this(bitmapFactory, utf8Dictionary, bitmaps, indexedTree, ColumnConfig.ALWAYS_USE_INDEXES, Integer.MAX_VALUE);
}
public StringUtf8ColumnIndexSupplier(
BitmapFactory bitmapFactory,
Supplier<TIndexed> utf8Dictionary,
@Nullable GenericIndexed<ImmutableBitmap> 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<TIndexed extends Indexed<ByteBuffer>>
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) ||

View File

@ -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);
}

View File

@ -98,6 +98,7 @@ public class LikeDimFilterTest extends InitializedNullHandlingTest
EqualsVerifier.forClass(LikeDimFilter.LikeMatcher.class)
.usingGetClass()
.withNonnullFields("suffixMatch", "prefix", "pattern")
.withIgnoredFields("likePattern")
.verify();
}

View File

@ -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)

View File

@ -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,

View File

@ -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)

View File

@ -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);
}
}

View File

@ -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(

View File

@ -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(

View File

@ -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()
);

View File

@ -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";

View File

@ -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<ImmutableBitmap> 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
);
}

View File

@ -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<String> failureReason = new AtomicReference<>(expectedReason);

View File

@ -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<String> failureReason = new AtomicReference<>(expectedReason);

View File

@ -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<String> failureReason = new AtomicReference<>(expectedReason);

View File

@ -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<String> failureReason = new AtomicReference<>(expectedReason);