remove search auto strategy, estimateSelectivity of BitmapColumnIndex (#15550)

* remove search auto strategy, estimateSelectivity of BitmapColumnIndex

* more cleanup
This commit is contained in:
Clint Wylie 2023-12-13 16:30:01 -08:00 committed by GitHub
parent f770eeb8be
commit e55f6b6202
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
47 changed files with 4 additions and 898 deletions

View File

@ -145,6 +145,5 @@ public class DictionaryEncodedStringIndexSupplierBenchmark
public void doValueSetCheck(Blackhole blackhole, BenchmarkState state)
{
BitmapColumnIndex bitmapIndex = state.stringValueSetIndex.forSortedValuesUtf8(state.values);
bitmapIndex.estimateSelectivity(10_000_000);
}
}

View File

@ -126,10 +126,6 @@ are unique.
queryableIndexSegment, and then evaluates search predicates. If some filters support bitmap indexes, the cursor can read
only the rows which satisfy those filters, thereby saving I/O cost. However, it might be slow with filters of low selectivity.
- "auto" strategy uses a cost-based planner for choosing an optimal search strategy. It estimates the cost of index-only
and cursor-based execution plans, and chooses the optimal one. Currently, it is not enabled by default due to the overhead
of cost estimation.
## Server configuration
The following runtime properties apply:

View File

@ -40,7 +40,6 @@ import org.apache.druid.segment.BaseFloatColumnValueSelector;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
@ -271,12 +270,6 @@ public class ArrayContainsElementFilter extends AbstractOptimizableDimFilter imp
).makeMatcher(predicateFactory);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector);
}
@Override
public boolean canVectorizeMatcher(ColumnInspector inspector)
{

View File

@ -19,7 +19,6 @@
package org.apache.druid.query.filter;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import java.util.HashSet;
@ -70,14 +69,4 @@ public interface BooleanFilter extends Filter
return allColumns;
}
@Override
default boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
for (Filter filter : getFilters()) {
if (!filter.supportsSelectivityEstimation(columnSelector, indexSelector)) {
return false;
}
}
return true;
}
}

View File

@ -44,7 +44,6 @@ import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnProcessorFactory;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
@ -267,12 +266,6 @@ public class EqualityFilter extends AbstractOptimizableDimFilter implements Filt
).makeMatcher(new EqualityPredicateFactory(matchValueEval));
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector);
}
@Override
public boolean canVectorizeMatcher(ColumnInspector inspector)
{

View File

@ -24,7 +24,6 @@ import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.BitmapResultFactory;
import org.apache.druid.query.filter.vector.VectorValueMatcher;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.index.BitmapColumnIndex;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
@ -70,37 +69,6 @@ public interface Filter
throw new UOE("Filter[%s] cannot vectorize", getClass().getName());
}
/**
* Estimate selectivity of this filter.
* This method can be used for cost-based query planning like in {@link org.apache.druid.query.search.AutoStrategy}.
* To avoid significant performance degradation for calculating the exact cost,
* implementation of this method targets to achieve rapid selectivity estimation
* with reasonable sacrifice of the accuracy.
* As a result, the estimated selectivity might be different from the exact value.
*
* @param indexSelector Object used to retrieve indexes
*
* @return an estimated selectivity ranging from 0 (filter selects no rows) to 1 (filter selects all rows).
*
* @see Filter#getBitmapColumnIndex(ColumnIndexSelector)
*/
default double estimateSelectivity(ColumnIndexSelector indexSelector)
{
return getBitmapColumnIndex(indexSelector).estimateSelectivity(indexSelector.getNumRows());
}
/**
* Indicates whether this filter supports selectivity estimation.
* A filter supports selectivity estimation if it supports bitmap index and
* the dimension which the filter evaluates does not have multi values.
*
* @param columnSelector Object to check the dimension has multi values.
* @param indexSelector Object used to retrieve bitmap indexes
*
* @return true if this Filter supports selectivity estimation, false otherwise.
*/
boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector);
/**
* Returns true if this filter can produce a vectorized matcher from its "makeVectorMatcher" method.
* @param inspector Supplies type information for the selectors this filter will match against

View File

@ -54,7 +54,6 @@ import org.apache.druid.query.lookup.LookupExtractionFn;
import org.apache.druid.query.lookup.LookupExtractor;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.DimensionHandlerUtils;
import org.apache.druid.segment.column.ColumnIndexSupplier;
@ -364,12 +363,6 @@ public class InDimFilter extends AbstractOptimizableDimFilter implements Filter
}
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, dimension, columnSelector, indexSelector);
}
@Override
public String toString()
{

View File

@ -35,7 +35,6 @@ 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.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.column.TypeSignature;
@ -158,12 +157,6 @@ public class NullFilter extends AbstractOptimizableDimFilter implements Filter
).makeMatcher(NullPredicateFactory.INSTANCE);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector);
}
@Override
public boolean canVectorizeMatcher(ColumnInspector inspector)
{

View File

@ -43,7 +43,6 @@ import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFac
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.column.ColumnType;
@ -358,12 +357,6 @@ public class RangeFilter extends AbstractOptimizableDimFilter implements Filter
).makeMatcher(getPredicateFactory());
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, column, columnSelector, indexSelector);
}
@Override
public boolean canVectorizeMatcher(ColumnInspector inspector)
{

View File

@ -1,134 +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.search;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.ColumnIndexSelector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorColumnIndexSelector;
import org.apache.druid.segment.DeprecatedQueryableIndexColumnSelector;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.index.semantic.DictionaryEncodedStringValueIndex;
import java.util.List;
public class AutoStrategy extends SearchStrategy
{
public static final String NAME = "auto";
private static final EmittingLogger log = new EmittingLogger(AutoStrategy.class);
public static AutoStrategy of(SearchQuery query)
{
return new AutoStrategy(query);
}
private AutoStrategy(SearchQuery query)
{
super(query);
}
@Override
public List<SearchQueryExecutor> getExecutionPlan(SearchQuery query, Segment segment)
{
final QueryableIndex index = segment.asQueryableIndex();
if (index != null) {
final ColumnSelector columnSelector = new DeprecatedQueryableIndexColumnSelector(index);
final ColumnIndexSelector selector = new ColumnSelectorColumnIndexSelector(
index.getBitmapFactoryForDimensions(),
query.getVirtualColumns(),
columnSelector
);
// Index-only plan is used only when any filter is not specified or the filter supports bitmap indexes.
//
// Note: if some filters support bitmap indexes but others are not, the current implementation always employs
// the cursor-based plan. This can be more optimized. One possible optimization is generating a bitmap index
// from the non-bitmap-support filters, and then use it to compute the filtered result by intersecting bitmaps.
if (filter == null || filter.supportsSelectivityEstimation(columnSelector, selector)) {
final List<DimensionSpec> dimsToSearch = getDimsToSearch(
index.getAvailableDimensions(),
query.getDimensions()
);
// Choose a search query execution strategy depending on the query.
// The costs of index-only plan and cursor-based plan can be computed like below.
//
// c_index = (total cardinality of all search dimensions) * (bitmap intersection cost)
// * (search predicate processing cost)
// c_cursor = (# of rows in a segment) * (filter selectivity) * (# of dimensions)
// * (search predicate processing cost)
final SearchQueryDecisionHelper helper = getDecisionHelper(index);
final double useIndexStrategyCost = helper.getBitmapIntersectCost() * computeTotalCard(index, dimsToSearch);
final double cursorOnlyStrategyCost = (filter == null ? 1. : filter.estimateSelectivity(selector))
* selector.getNumRows()
* dimsToSearch.size();
log.debug(
"Use-index strategy cost: %f, cursor-only strategy cost: %f",
useIndexStrategyCost,
cursorOnlyStrategyCost
);
if (useIndexStrategyCost < cursorOnlyStrategyCost) {
log.debug("Use-index execution strategy is selected, query id [%s]", query.getId());
return UseIndexesStrategy.of(query).getExecutionPlan(query, segment);
} else {
log.debug("Cursor-only execution strategy is selected, query id [%s]", query.getId());
return CursorOnlyStrategy.of(query).getExecutionPlan(query, segment);
}
} else {
log.debug(
"Filter doesn't support bitmap index. Fall back to cursor-only execution strategy, query id [%s]",
query.getId()
);
return CursorOnlyStrategy.of(query).getExecutionPlan(query, segment);
}
} else {
log.debug("Index doesn't exist. Fall back to cursor-only execution strategy, query id [%s]", query.getId());
return CursorOnlyStrategy.of(query).getExecutionPlan(query, segment);
}
}
private static long computeTotalCard(final QueryableIndex index, final Iterable<DimensionSpec> dimensionSpecs)
{
long totalCard = 0;
for (DimensionSpec dimension : dimensionSpecs) {
final ColumnHolder columnHolder = index.getColumnHolder(dimension.getDimension());
if (columnHolder != null) {
final ColumnIndexSupplier indexSupplier = columnHolder.getIndexSupplier();
if (indexSupplier != null) {
final DictionaryEncodedStringValueIndex bitmapIndex =
indexSupplier.as(DictionaryEncodedStringValueIndex.class);
if (bitmapIndex != null) {
totalCard += bitmapIndex.getCardinality();
}
}
}
}
return totalCard;
}
}

View File

@ -1,38 +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.search;
public class ConciseBitmapDecisionHelper extends SearchQueryDecisionHelper
{
// This value comes from an experiment.
// See the discussion at https://github.com/apache/druid/pull/3792#issuecomment-268331804.
private static final double BITMAP_INTERSECT_COST = 7.425;
private static final ConciseBitmapDecisionHelper INSTANCE = new ConciseBitmapDecisionHelper();
public static ConciseBitmapDecisionHelper instance()
{
return INSTANCE;
}
private ConciseBitmapDecisionHelper()
{
super(BITMAP_INTERSECT_COST);
}
}

View File

@ -1,38 +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.search;
public class RoaringBitmapDecisionHelper extends SearchQueryDecisionHelper
{
// This value comes from an experiment.
// See the discussion at https://github.com/apache/druid/pull/3792#issuecomment-268331804.
private static final double BITMAP_INTERSECT_COST = 4.5;
private static final RoaringBitmapDecisionHelper INSTANCE = new RoaringBitmapDecisionHelper();
public static RoaringBitmapDecisionHelper instance()
{
return INSTANCE;
}
private RoaringBitmapDecisionHelper()
{
super(BITMAP_INTERSECT_COST);
}
}

View File

@ -1,35 +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.search;
public abstract class SearchQueryDecisionHelper
{
private final double bitmapIntersectCost;
protected SearchQueryDecisionHelper(final double bitmapIntersectCost)
{
this.bitmapIntersectCost = bitmapIntersectCost;
}
public double getBitmapIntersectCost()
{
return bitmapIntersectCost;
}
}

View File

@ -21,14 +21,10 @@ package org.apache.druid.query.search;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Iterables;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.collections.bitmap.ConciseBitmapFactory;
import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.query.Druids;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.filter.Filters;
@ -53,18 +49,6 @@ public abstract class SearchStrategy
public abstract List<SearchQueryExecutor> getExecutionPlan(SearchQuery query, Segment segment);
public SearchQueryDecisionHelper getDecisionHelper(QueryableIndex index)
{
final BitmapFactory bitmapFactory = index.getBitmapFactoryForDimensions();
if (bitmapFactory.getClass().equals(ConciseBitmapFactory.class)) {
return ConciseBitmapDecisionHelper.instance();
} else if (bitmapFactory.getClass().equals(RoaringBitmapFactory.class)) {
return RoaringBitmapDecisionHelper.instance();
} else {
throw new IAE("Unknown bitmap type[%s]", bitmapFactory.getClass().getName());
}
}
static List<DimensionSpec> getDimsToSearch(Indexed<String> availableDimensions, List<DimensionSpec> dimensions)
{
if (dimensions == null || dimensions.isEmpty()) {

View File

@ -40,9 +40,8 @@ public class SearchStrategySelector
final String strategyString = config.withOverrides(query).getSearchStrategy();
switch (strategyString) {
case AutoStrategy.NAME:
log.debug("Auto strategy is selected, query id [%s]", query.getId());
return AutoStrategy.of(query);
case "auto":
log.debug("Auto strategy is selected but has been removed, using 'use-index' strategy instead for query id [%s]", query.getId());
case UseIndexesStrategy.NAME:
log.debug("Use-index strategy is selected, query id [%s]", query.getId());
return UseIndexesStrategy.of(query);

View File

@ -114,17 +114,6 @@ public class AndFilter implements BooleanFilter
return finalMerged;
}
@Override
public double estimateSelectivity(int totalRows)
{
// Estimate selectivity with attribute value independence assumption
double selectivity = 1.0;
for (final Filter filter : filters) {
selectivity *= filter.estimateSelectivity(selector);
}
return selectivity;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{

View File

@ -42,7 +42,6 @@ import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFac
import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexCapabilities;
import org.apache.druid.segment.column.ColumnIndexSupplier;
@ -169,15 +168,6 @@ public class BoundFilter implements Filter
return rangeIndex.getIndexCapabilities().merge(nullBitmap.getIndexCapabilities());
}
@Override
public double estimateSelectivity(int totalRows)
{
return Math.min(
1.0,
rangeIndex.estimateSelectivity(totalRows) + nullBitmap.estimateSelectivity(totalRows)
);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
@ -225,12 +215,6 @@ public class BoundFilter implements Filter
return true;
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, boundDimFilter.getDimension(), columnSelector, indexSelector);
}
@Override
public Set<String> getRequiredColumns()
{

View File

@ -32,7 +32,6 @@ import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.BaseObjectColumnValueSelector;
import org.apache.druid.segment.ColumnProcessorFactory;
import org.apache.druid.segment.ColumnProcessors;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
@ -141,24 +140,12 @@ public class ColumnComparisonFilter implements Filter
return false;
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return false;
}
@Override
public Set<String> getRequiredColumns()
{
return dimensions.stream().map(DimensionSpec::getDimension).collect(Collectors.toSet());
}
@Override
public double estimateSelectivity(ColumnIndexSelector indexSelector)
{
throw new UnsupportedOperationException();
}
@Override
public boolean equals(Object o)
{

View File

@ -37,7 +37,6 @@ 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.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.index.BitmapColumnIndex;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
@ -124,12 +123,6 @@ public class DimensionPredicateFilter implements Filter
return ImmutableSet.of(dimension);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, dimension, columnSelector, indexSelector);
}
@Override
public String toString()
{

View File

@ -42,7 +42,6 @@ import org.apache.druid.query.filter.vector.VectorValueMatcher;
import org.apache.druid.query.filter.vector.VectorValueMatcherColumnProcessorFactory;
import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.column.ColumnCapabilities;
@ -231,22 +230,6 @@ public class ExpressionFilter implements Filter
return null;
}
@Override
public boolean supportsSelectivityEstimation(
final ColumnSelector columnSelector,
final ColumnIndexSelector indexSelector
)
{
return false;
}
@Override
public double estimateSelectivity(final ColumnIndexSelector indexSelector)
{
// Selectivity estimation not supported.
throw new UnsupportedOperationException();
}
@Override
public Set<String> getRequiredColumns()
{

View File

@ -24,7 +24,6 @@ import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.filter.vector.VectorValueMatcher;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.index.AllFalseBitmapColumnIndex;
import org.apache.druid.segment.index.BitmapColumnIndex;
@ -55,12 +54,6 @@ public class FalseFilter implements Filter
return new AllFalseBitmapColumnIndex(selector.getBitmapFactory());
}
@Override
public double estimateSelectivity(ColumnIndexSelector indexSelector)
{
return 0;
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
@ -73,12 +66,6 @@ public class FalseFilter implements Filter
return ConstantMatcherType.ALL_FALSE.asVectorMatcher(factory.getReadableVectorInspector());
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return true;
}
@Override
public boolean canVectorizeMatcher(ColumnInspector inspector)
{

View File

@ -34,9 +34,7 @@ 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.segment.ColumnProcessors;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.filter.cnf.CNFFilterExplosionException;
import org.apache.druid.segment.filter.cnf.CalciteCnfHelper;
@ -52,7 +50,6 @@ import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Objects;
@ -158,36 +155,6 @@ public class Filters
);
}
public static boolean supportsSelectivityEstimation(
Filter filter,
String dimension,
ColumnSelector columnSelector,
ColumnIndexSelector indexSelector
)
{
if (filter.getBitmapColumnIndex(indexSelector) != null) {
final ColumnHolder columnHolder = columnSelector.getColumnHolder(dimension);
if (columnHolder != null) {
return columnHolder.getCapabilities().hasMultipleValues().isFalse();
}
}
return false;
}
public static double estimateSelectivity(
final Iterator<ImmutableBitmap> bitmaps,
final long totalNumRows
)
{
long numMatchedRows = 0;
while (bitmaps.hasNext()) {
final ImmutableBitmap bitmap = bitmaps.next();
numMatchedRows += bitmap.size();
}
return Math.min(1, (double) numMatchedRows / totalNumRows);
}
@Nullable
public static Filter convertToCNFFromQueryContext(Query query, @Nullable Filter filter)
{

View File

@ -31,7 +31,6 @@ 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.ColumnInspector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexCapabilities;
import org.apache.druid.segment.index.BitmapColumnIndex;
@ -79,12 +78,6 @@ public class IsBooleanFilter implements Filter
return baseIndex.getIndexCapabilities();
}
@Override
public double estimateSelectivity(int totalRows)
{
return 1. - baseFilter.estimateSelectivity(selector);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
@ -176,12 +169,6 @@ public class IsBooleanFilter implements Filter
return new IsBooleanFilter(baseFilter.rewriteRequiredColumns(columnRewrites), isTrue);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return baseFilter.supportsSelectivityEstimation(columnSelector, indexSelector);
}
@Override
public String toString()
{

View File

@ -25,7 +25,6 @@ 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.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.index.BitmapColumnIndex;
@ -67,12 +66,6 @@ public class JavaScriptFilter implements Filter
return Filters.makeValueMatcher(factory, dimension, predicateFactory);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, dimension, columnSelector, indexSelector);
}
@Override
public Set<String> getRequiredColumns()
{

View File

@ -32,7 +32,6 @@ 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.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
@ -166,12 +165,6 @@ public class LikeFilter implements Filter
);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, dimension, columnSelector, indexSelector);
}
/**
* Returns true if this filter is a simple equals filter: dimension = 'value' with no extractionFn.
*/

View File

@ -32,7 +32,6 @@ 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.ColumnInspector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexCapabilities;
import org.apache.druid.segment.index.BitmapColumnIndex;
@ -81,12 +80,6 @@ public class NotFilter implements Filter
return baseIndex.getIndexCapabilities();
}
@Override
public double estimateSelectivity(int totalRows)
{
return 1. - baseFilter.estimateSelectivity(selector);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
@ -169,12 +162,6 @@ public class NotFilter implements Filter
return new NotFilter(baseFilter.rewriteRequiredColumns(columnRewrites));
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return baseFilter.supportsSelectivityEstimation(columnSelector, indexSelector);
}
@Override
public String toString()
{

View File

@ -100,17 +100,6 @@ public class OrFilter implements BooleanFilter
return finalMerged;
}
@Override
public double estimateSelectivity(int totalRows)
{
// Estimate selectivity with attribute value independence assumption
double selectivity = 0;
for (final Filter filter : filters) {
selectivity += filter.estimateSelectivity(selector);
}
return Math.min(selectivity, 1.);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{

View File

@ -31,7 +31,6 @@ 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.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.index.BitmapColumnIndex;
@ -125,12 +124,6 @@ public class SelectorFilter implements Filter
).makeMatcher(value);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return Filters.supportsSelectivityEstimation(this, dimension, columnSelector, indexSelector);
}
@Override
public boolean canVectorizeMatcher(ColumnInspector inspector)
{

View File

@ -34,7 +34,6 @@ import org.apache.druid.query.filter.DruidPredicateFactory;
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.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.column.ColumnIndexCapabilities;
import org.apache.druid.segment.column.ColumnIndexSupplier;
@ -90,13 +89,6 @@ public class SpatialFilter implements Filter
return new SimpleColumnIndexCapabilities(true, true);
}
@Override
public double estimateSelectivity(int totalRows)
{
// selectivity estimation for multi-value columns is not implemented yet.
throw new UnsupportedOperationException();
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{
@ -117,25 +109,12 @@ public class SpatialFilter implements Filter
);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return false;
}
@Override
public Set<String> getRequiredColumns()
{
return ImmutableSet.of(dimension);
}
@Override
public double estimateSelectivity(ColumnIndexSelector indexSelector)
{
// selectivity estimation for multi-value columns is not implemented yet.
throw new UnsupportedOperationException();
}
@Override
public boolean equals(Object o)
{

View File

@ -24,7 +24,6 @@ import org.apache.druid.query.filter.Filter;
import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.filter.vector.VectorValueMatcher;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.index.AllTrueBitmapColumnIndex;
import org.apache.druid.segment.index.BitmapColumnIndex;
@ -70,12 +69,6 @@ public class TrueFilter implements Filter
return ConstantMatcherType.ALL_TRUE.asVectorMatcher(factory.getReadableVectorInspector());
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return true;
}
@Override
public boolean canVectorizeMatcher(ColumnInspector inspector)
{
@ -100,12 +93,6 @@ public class TrueFilter implements Filter
return this;
}
@Override
public double estimateSelectivity(ColumnIndexSelector indexSelector)
{
return 1;
}
@Override
public String toString()
{

View File

@ -50,12 +50,6 @@ public class AllFalseBitmapColumnIndex implements BitmapColumnIndex
return SimpleColumnIndexCapabilities.getConstant();
}
@Override
public double estimateSelectivity(int totalRows)
{
return 0;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{

View File

@ -39,12 +39,6 @@ public class AllTrueBitmapColumnIndex implements BitmapColumnIndex
return SimpleColumnIndexCapabilities.getConstant();
}
@Override
public double estimateSelectivity(int totalRows)
{
return 1;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{

View File

@ -44,12 +44,6 @@ public class AllUnknownBitmapColumnIndex implements BitmapColumnIndex
return SimpleColumnIndexCapabilities.getConstant();
}
@Override
public double estimateSelectivity(int totalRows)
{
return 0;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{

View File

@ -30,8 +30,6 @@ public interface BitmapColumnIndex
{
ColumnIndexCapabilities getIndexCapabilities();
double estimateSelectivity(int totalRows);
/**
* Compute a bitmap result wrapped with the {@link BitmapResultFactory} representing the rows matched by this index.
*

View File

@ -82,11 +82,6 @@ public final class IndexedUtf8ValueIndexes<TDictionary extends Indexed<ByteBuffe
final ByteBuffer utf8 = StringUtils.toUtf8ByteBuffer(value);
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
return Math.min(1, (double) getBitmapForValue().size() / totalRows);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)

View File

@ -35,12 +35,6 @@ public final class SimpleImmutableBitmapIndex extends SimpleBitmapColumnIndex
this.bitmap = bitmap;
}
@Override
public double estimateSelectivity(int totalRows)
{
return Math.min(1, (double) bitmap.size() / totalRows);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
{

View File

@ -22,7 +22,6 @@ package org.apache.druid.segment.index;
import com.google.common.collect.Iterables;
import org.apache.druid.collections.bitmap.ImmutableBitmap;
import org.apache.druid.query.BitmapResultFactory;
import org.apache.druid.segment.filter.Filters;
import javax.annotation.Nullable;
import java.util.Collections;
@ -32,11 +31,6 @@ import java.util.Collections;
*/
public abstract class SimpleImmutableBitmapIterableIndex extends SimpleBitmapColumnIndex
{
@Override
public double estimateSelectivity(int totalRows)
{
return Filters.estimateSelectivity(getBitmapIterable().iterator(), totalRows);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)

View File

@ -374,15 +374,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
{
final FixedIndexed<Integer> localDictionary = localDictionarySupplier.get();
final Indexed<ByteBuffer> stringDictionary = globalStringDictionarySupplier.get();
@Override
public double estimateSelectivity(int totalRows)
{
final int globalId = stringDictionary.indexOf(StringUtils.toUtf8ByteBuffer(value));
if (globalId < 0) {
return 0.0;
}
return (double) getBitmap(localDictionary.indexOf(globalId)).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
@ -664,22 +655,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
final FixedIndexed<Integer> localDictionary = localDictionarySupplier.get();
final FixedIndexed<Long> longDictionary = globalLongDictionarySupplier.get();
@Override
public double estimateSelectivity(int totalRows)
{
if (longValue == null) {
if (inputNull) {
return (double) getBitmap(localDictionary.indexOf(0)).size() / totalRows;
} else {
return 0.0;
}
}
final int globalId = longDictionary.indexOf(longValue);
if (globalId < 0) {
return 0.0;
}
return (double) getBitmap(localDictionary.indexOf(globalId + adjustLongId)).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
@ -924,22 +899,6 @@ public class NestedFieldColumnIndexSupplier<TStringDictionary extends Indexed<By
{
final FixedIndexed<Integer> localDictionary = localDictionarySupplier.get();
final FixedIndexed<Double> doubleDictionary = globalDoubleDictionarySupplier.get();
@Override
public double estimateSelectivity(int totalRows)
{
if (doubleValue == null) {
if (inputNull) {
return (double) getBitmap(localDictionary.indexOf(0)).size() / totalRows;
} else {
return 0.0;
}
}
final int globalId = doubleDictionary.indexOf(doubleValue);
if (globalId < 0) {
return 0.0;
}
return (double) getBitmap(localDictionary.indexOf(globalId + adjustDoubleId)).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)

View File

@ -242,15 +242,6 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
return new SimpleBitmapColumnIndex()
{
final FixedIndexed<Double> dictionary = doubleDictionarySupplier.get();
@Override
public double estimateSelectivity(int totalRows)
{
final int id = dictionary.indexOf(doubleValue);
if (id < 0) {
return 0.0;
}
return (double) getBitmap(id).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
@ -285,28 +276,6 @@ public class ScalarDoubleColumnAndIndexSupplier implements Supplier<NestedCommon
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
if (doubleValue == null) {
if (inputNull && NullHandling.sqlCompatible()) {
return (double) getBitmap(0).size() / totalRows;
} else {
return 0.0;
}
}
if (NullHandling.replaceWithDefault() && doubleValue.equals(NullHandling.defaultDoubleValue())) {
if (defaultValueIndex >= 0) {
return ((double) getBitmap(0).size() + (double) getBitmap(defaultValueIndex).size()) / totalRows;
}
return (double) getBitmap(0).size() / totalRows;
}
final int id = dictionary.indexOf(doubleValue);
if (id < 0) {
return 0.0;
}
return (double) getBitmap(id).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)

View File

@ -243,15 +243,6 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
return new SimpleBitmapColumnIndex()
{
final FixedIndexed<Long> dictionary = longDictionarySupplier.get();
@Override
public double estimateSelectivity(int totalRows)
{
final int id = dictionary.indexOf(longValue);
if (id < 0) {
return 0.0;
}
return (double) getBitmap(id).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
@ -286,28 +277,6 @@ public class ScalarLongColumnAndIndexSupplier implements Supplier<NestedCommonFo
final Long longValue = GuavaUtils.tryParseLong(value);
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
if (longValue == null) {
if (inputNull && NullHandling.sqlCompatible()) {
return (double) getBitmap(0).size() / totalRows;
} else {
return 0.0;
}
}
if (NullHandling.replaceWithDefault() && longValue.equals(NullHandling.defaultLongValue())) {
if (defaultValueIndex >= 0) {
return ((double) getBitmap(0).size() + (double) getBitmap(defaultValueIndex).size()) / totalRows;
}
return (double) getBitmap(0).size() / totalRows;
}
final int id = dictionary.indexOf(longValue);
if (id < 0) {
return 0.0;
}
return (double) getBitmap(id).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)

View File

@ -374,15 +374,6 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
final FrontCodedIntArrayIndexed dictionary = arrayDictionarySupplier.get();
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
final int id = dictionary.indexOf(ids) + arrayOffset;
if (id < 0) {
return 0.0;
}
return (double) getBitmap(id).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
@ -448,15 +439,6 @@ public class VariantColumnAndIndexSupplier implements Supplier<NestedCommonForma
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
final int elementId = getElementId();
if (elementId < 0) {
return 0.0;
}
return (double) getElementBitmap(elementId).size() / totalRows;
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)

View File

@ -49,7 +49,6 @@ import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnCapabilitiesImpl;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnIndexSupplier;
import org.apache.druid.segment.filter.Filters;
import org.apache.druid.segment.index.BitmapColumnIndex;
import org.apache.druid.segment.index.SimpleBitmapColumnIndex;
import org.apache.druid.segment.index.SimpleImmutableBitmapIterableIndex;
@ -415,14 +414,6 @@ public class ListFilteredVirtualColumn implements VirtualColumn
{
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
return Filters.estimateSelectivity(
Collections.singletonList(nullValueBitmapSupplier.get()).iterator(),
totalRows
);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknowns)
@ -453,11 +444,6 @@ public class ListFilteredVirtualColumn implements VirtualColumn
{
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
return Math.min(1, (double) getBitmapForValue().size() / totalRows);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)
@ -558,15 +544,6 @@ public class ListFilteredVirtualColumn implements VirtualColumn
{
return new SimpleBitmapColumnIndex()
{
@Override
public double estimateSelectivity(int totalRows)
{
final int start = 0, end = getCardinality();
return Filters.estimateSelectivity(
getBitmapsInRange(matcherFactory.makeStringPredicate(), start, end).iterator(),
totalRows
);
}
@Override
public <T> T computeBitmapResult(BitmapResultFactory<T> bitmapResultFactory, boolean includeUnknown)

View File

@ -62,8 +62,9 @@ public class SearchQueryRunnerWithCaseTest extends InitializedNullHandlingTest
configs[0].setSearchStrategy(UseIndexesStrategy.NAME);
configs[1] = new SearchQueryConfig();
configs[1].setSearchStrategy(CursorOnlyStrategy.NAME);
// test auto to ensure that it doesn't explode
configs[2] = new SearchQueryConfig();
configs[2].setSearchStrategy(AutoStrategy.NAME);
configs[2].setSearchStrategy("auto");
CharSource input = CharSource.wrap(
"2011-01-12T00:00:00.000Z\tspot\tAutoMotive\t1000\t10000.0\t10000.0\t100000\t10\t10.0\t10.0\tPREFERRED\ta\u0001preferred\t100.000000\n" +

View File

@ -67,7 +67,6 @@ import org.apache.druid.query.filter.ValueMatcher;
import org.apache.druid.query.filter.vector.VectorValueMatcher;
import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionSelector;
@ -821,24 +820,12 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
return theFilter.makeMatcher(factory);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return false;
}
@Override
public Set<String> getRequiredColumns()
{
return Collections.emptySet();
}
@Override
public double estimateSelectivity(ColumnIndexSelector indexSelector)
{
return 1.0;
}
@Nullable
@Override
public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)
@ -903,18 +890,6 @@ public abstract class BaseFilterTest extends InitializedNullHandlingTest
return null;
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return false;
}
@Override
public double estimateSelectivity(ColumnIndexSelector indexSelector)
{
return 1.0;
}
@Nullable
@Override
public BitmapColumnIndex getBitmapColumnIndex(ColumnIndexSelector selector)

View File

@ -56,7 +56,6 @@ import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.query.topn.TopNQueryEngine;
import org.apache.druid.query.topn.TopNResultValue;
import org.apache.druid.segment.CloserRule;
import org.apache.druid.segment.ColumnSelector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionSelector;
@ -639,12 +638,6 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
return new AllTrueBitmapColumnIndex(selector);
}
@Override
public double estimateSelectivity(ColumnIndexSelector indexSelector)
{
return 1;
}
@Override
public ValueMatcher makeMatcher(ColumnSelectorFactory factory)
{
@ -655,12 +648,6 @@ public class IncrementalIndexStorageAdapterTest extends InitializedNullHandlingT
);
}
@Override
public boolean supportsSelectivityEstimation(ColumnSelector columnSelector, ColumnIndexSelector indexSelector)
{
return true;
}
@Override
public Set<String> getRequiredColumns()
{

View File

@ -170,7 +170,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = nullIndex.get();
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.0, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
Assert.assertEquals(0, bitmap.size());
}
@ -189,21 +188,18 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("b");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 7, 8);
// non-existent in local column
columnIndex = valueSetIndex.forValue("fo");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.0, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
// set index
columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("b", "fooo", "z")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.8, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 2, 3, 4, 5, 6, 7, 8);
}
@ -223,144 +219,118 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex forRange = rangeIndex.forRange(null, false, "a", false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange(null, true, "a", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange(null, false, "b", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange(null, false, "b", false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 7, 8);
forRange = rangeIndex.forRange("a", false, "b", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange("a", true, "b", false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 7, 8);
forRange = rangeIndex.forRange("b", false, "fon", false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 7, 8);
forRange = rangeIndex.forRange("bb", false, "fon", false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange("b", true, "foo", false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 9);
forRange = rangeIndex.forRange("f", true, "g", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 5, 9);
forRange = rangeIndex.forRange(null, false, "g", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 5, 7, 8, 9);
forRange = rangeIndex.forRange("f", false, null, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 5, 6, 9);
forRange = rangeIndex.forRange("b", true, "fooo", true);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 9);
forRange = rangeIndex.forRange("b", true, "fooo", false);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 5, 9);
forRange = rangeIndex.forRange(null, true, "fooo", true);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 3, 7, 8, 9);
forRange = rangeIndex.forRange("b", true, null, false);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 5, 6, 9);
forRange = rangeIndex.forRange("b", false, null, true);
Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
forRange = rangeIndex.forRange(null, true, "fooo", false);
Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 5, 7, 8, 9);
forRange = rangeIndex.forRange(null, true, null, true);
Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
forRange = rangeIndex.forRange(null, false, null, false);
Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
forRange = rangeIndex.forRange(null, true, "foa", false);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 7, 8);
forRange = rangeIndex.forRange(null, true, "foooa", false);
Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 5, 7, 8, 9);
forRange = rangeIndex.forRange("foooa", true, "ggg", false);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange("g", true, "gg", false);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange("z", true, "zz", false);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange("z", false, "zz", false);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 4, 6);
}
@ -384,7 +354,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
true,
s -> !"fooo".equals(s)
);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 9);
@ -395,7 +364,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
true,
s -> "fooo".equals(s)
);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 5);
@ -406,7 +374,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
false,
s -> !"fooo".equals(s)
);
Assert.assertEquals(0.8, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 8, 9);
@ -417,7 +384,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
true,
s -> !"fooo".equals(s)
);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 3, 7, 8, 9);
@ -428,7 +394,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
true,
s -> true
);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 5, 6, 9);
}
@ -451,7 +416,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 4, 6, 7, 8);
}
@ -470,7 +434,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = nullIndex.get();
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 7, 8);
}
@ -489,21 +452,18 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("b");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 3);
// non-existent in local column
columnIndex = valueSetIndex.forValue("fo");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.0, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
// set index
columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("b", "fooo", "z")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 3, 4, 5, 6);
}
@ -522,78 +482,64 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex forRange = rangeIndex.forRange("f", true, "g", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 5, 9);
forRange = rangeIndex.forRange(null, false, "g", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 3, 5, 9);
forRange = rangeIndex.forRange(null, false, "a", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange(null, false, "b", true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndex.forRange(null, false, "b", false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 3);
forRange = rangeIndex.forRange("f", false, null, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 5, 6, 9);
forRange = rangeIndex.forRange("b", true, "fooo", true);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 9);
forRange = rangeIndex.forRange("b", true, "fooo", false);
Assert.assertEquals(0.4, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 5, 9);
forRange = rangeIndex.forRange(null, true, "fooo", true);
Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 3, 9);
forRange = rangeIndex.forRange("b", true, null, false);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 5, 6, 9);
forRange = rangeIndex.forRange("b", false, null, true);
Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 3, 4, 5, 6, 9);
forRange = rangeIndex.forRange(null, true, "fooo", false);
Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 3, 5, 9);
forRange = rangeIndex.forRange(null, true, null, true);
Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 3, 4, 5, 6, 9);
forRange = rangeIndex.forRange(null, false, null, false);
Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 3, 4, 5, 6, 9);
}
@ -616,7 +562,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 3, 4, 6);
}
@ -638,14 +583,12 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("1");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 9);
// set index
columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1", "300", "700")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 2, 3, 7, 8, 9);
}
@ -664,57 +607,47 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex forRange = rangeIndexes.forRange(10L, true, 400L, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 6, 7, 8);
forRange = rangeIndexes.forRange(1, true, 3, true);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(1, false, 3, true);
Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 9);
forRange = rangeIndexes.forRange(1, false, 3, false);
Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 4, 5, 9);
forRange = rangeIndexes.forRange(100L, true, 300L, true);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(100L, true, 300L, false);
Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 7, 8);
forRange = rangeIndexes.forRange(100L, false, 300L, true);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 6);
forRange = rangeIndexes.forRange(100L, false, 300L, false);
Assert.assertEquals(0.5, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 6, 7, 8);
forRange = rangeIndexes.forRange(null, true, null, true);
Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
forRange = rangeIndexes.forRange(null, false, null, false);
Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
}
@ -737,7 +670,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 4, 5, 9);
}
@ -756,7 +688,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = nullIndex.get();
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 5, 8);
}
@ -775,14 +706,12 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("3");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 4);
// set index
columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1", "3", "300")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 4, 7, 9);
@ -794,14 +723,12 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
treeSet.add("300");
columnIndex = valueSetIndex.forSortedValues(treeSet);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.8, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 2, 3, 4, 5, 7, 8, 9);
// null value should really use NullValueIndex, but this works for classic reasons
columnIndex = valueSetIndex.forValue(null);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 5, 8);
}
@ -820,53 +747,43 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex forRange = rangeIndexes.forRange(100, false, 700, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 6, 7);
forRange = rangeIndexes.forRange(100, true, 300, true);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(100, false, 300, true);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 6);
forRange = rangeIndexes.forRange(100, true, 300, false);
Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 7);
forRange = rangeIndexes.forRange(100, false, 300, false);
Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 6, 7);
forRange = rangeIndexes.forRange(null, true, null, true);
Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 9);
forRange = rangeIndexes.forRange(null, false, null, false);
Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 3, 4, 6, 7, 9);
forRange = rangeIndexes.forRange(null, false, 0, false);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(null, false, 1, false);
Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 9);
forRange = rangeIndexes.forRange(null, false, 1, true);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
}
@ -889,7 +806,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 4, 6);
}
@ -911,14 +827,12 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("1.2");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 4, 7);
// set index
columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1.2", "3.3", "6.6")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.7, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 3, 4, 5, 6, 7, 9);
}
@ -937,80 +851,68 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex forRange = rangeIndexes.forRange(1.0, true, 5.0, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.9, forRange.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 6, 7, 8, 9);
forRange = rangeIndexes.forRange(1.1, false, 3.3, false);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.9, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 6, 7, 8, 9);
forRange = rangeIndexes.forRange(1.1, true, 3.3, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.3, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 4, 7);
forRange = rangeIndexes.forRange(null, true, null, true);
Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
forRange = rangeIndexes.forRange(null, false, null, false);
Assert.assertEquals(1.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
forRange = rangeIndexes.forRange(1.111, true, 1.19, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(1.01, true, 1.09, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(0.05, true, 0.98, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(0.05, true, 1.1, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(8.99, true, 10.10, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(8.99, true, 10.10, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(10.00, true, 10.10, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
@ -1034,7 +936,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 3, 4, 6, 7, 9);
}
@ -1053,7 +954,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = nullIndex.get();
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 6);
}
@ -1072,14 +972,12 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("6.6");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 5);
// set index
columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("1.2", "3.3", "7.7")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 4, 7, 9);
@ -1091,14 +989,12 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
treeSet.add("7.7");
columnIndex = valueSetIndex.forSortedValues(treeSet);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.7, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 2, 3, 4, 6, 7, 9);
// null value should really use NullValueIndex, but this works for classic reasons
columnIndex = valueSetIndex.forValue(null);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 6);
}
@ -1117,38 +1013,31 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex forRange = rangeIndexes.forRange(1.1, false, 5.0, true);
Assert.assertNotNull(forRange);
Assert.assertEquals(0.6, forRange.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 7, 8, 9);
forRange = rangeIndexes.forRange(null, true, null, true);
Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 5, 7, 8, 9);
forRange = rangeIndexes.forRange(null, false, null, false);
Assert.assertEquals(0.7, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 2, 4, 5, 7, 8, 9);
forRange = rangeIndexes.forRange(null, true, 1.0, true);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
forRange = rangeIndexes.forRange(null, true, 1.1, false);
Assert.assertEquals(0.2, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 8);
forRange = rangeIndexes.forRange(6.6, false, null, false);
Assert.assertEquals(0.1, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 5);
forRange = rangeIndexes.forRange(6.6, true, null, false);
Assert.assertEquals(0.0, forRange.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = forRange.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
}
@ -1171,7 +1060,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 4, 7, 9);
}
@ -1193,7 +1081,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = nullIndex.get();
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 7);
}
@ -1212,26 +1099,22 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("b");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 9);
columnIndex = valueSetIndex.forValue("1");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 5);
columnIndex = valueSetIndex.forValue("1.1");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.1, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 8);
// set index
columnIndex = valueSetIndex.forSortedValues(new TreeSet<>(ImmutableSet.of("b", "300", "9.9", "1.6")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.4, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 4, 9);
@ -1244,14 +1127,12 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
treeSet.add("1.6");
columnIndex = valueSetIndex.forSortedValues(treeSet);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.6, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 2, 3, 4, 7, 9);
// null value should really use NullValueIndex, but this works for classic reasons
columnIndex = valueSetIndex.forValue(null);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.2, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 7);
}
@ -1286,7 +1167,6 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = predicateIndex.forPredicate(predicateFactory);
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.5, columnIndex.estimateSelectivity(ROW_COUNT), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 1, 3, 4, 6, 9);
}
@ -1442,19 +1322,16 @@ public class NestedFieldColumnIndexSupplierTest extends InitializedNullHandlingT
BitmapColumnIndex columnIndex = valueSetIndex.forValue("1");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3333, columnIndex.estimateSelectivity(3), 0.001);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0);
columnIndex = valueSetIndex.forValue("-2");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.3333, columnIndex.estimateSelectivity(3), 0.001);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2);
columnIndex = valueSetIndex.forValue("2");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.0, columnIndex.estimateSelectivity(3), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
}

View File

@ -62,42 +62,36 @@ public class DictionaryEncodedStringIndexSupplierTest extends InitializedNullHan
BitmapColumnIndex columnIndex = valueSetIndex.forValue("b");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.1, columnIndex.estimateSelectivity(10), 0.0);
ImmutableBitmap bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 3);
// non-existent in local column
columnIndex = valueSetIndex.forValue("fo");
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.0, columnIndex.estimateSelectivity(10), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
// set index
columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("b", "fooo", "z")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.5, columnIndex.estimateSelectivity(10), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 2, 3, 4, 5, 6);
// set index with single value in middle
columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("foo")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.2, columnIndex.estimateSelectivity(10), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap, 0, 9);
// set index with no element in column and all elements less than lowest non-null value
columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("a", "aa", "aaa")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.0, columnIndex.estimateSelectivity(10), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
// set index with no element in column and all elements greater than highest non-null value
columnIndex = valueSetIndex.forSortedValues(InDimFilter.ValuesSet.copyOf(ImmutableSet.of("zz", "zzz", "zzzz")));
Assert.assertNotNull(columnIndex);
Assert.assertEquals(0.0, columnIndex.estimateSelectivity(10), 0.0);
bitmap = columnIndex.computeBitmapResult(bitmapResultFactory, false);
checkBitmap(bitmap);
}