mirror of
https://github.com/apache/druid.git
synced 2025-02-16 23:15:16 +00:00
remove search auto strategy, estimateSelectivity of BitmapColumnIndex (#15550)
* remove search auto strategy, estimateSelectivity of BitmapColumnIndex * more cleanup
This commit is contained in:
parent
f770eeb8be
commit
e55f6b6202
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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:
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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()) {
|
||||
|
@ -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);
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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.
|
||||
*/
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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.
|
||||
*
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
|
@ -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" +
|
||||
|
@ -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)
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user