mirror of https://github.com/apache/druid.git
Add an option to SearchQuery to choose a search query execution strategy (#3792)
* Add an option to SearchQuery to choose a search query execution strategy. Supported strategies are 1) Index-only query execution 2) Cursor-based scan 3) Auto: choose an efficient strategy for a given query * Add SearchStrategy and SearchQueryExecutor * Address comments * Rename strategies and set UseIndexesStrategy as the default strategy * Add a cost-based planner for auto strategy * Add document * Fix code style * apply code style * apply comments
This commit is contained in:
parent
ce0049d8ff
commit
c099977a5b
|
@ -45,7 +45,7 @@ public class BenchmarkSchemas
|
|||
// dims
|
||||
BenchmarkColumnSchema.makeSequential("dimSequential", ValueType.STRING, false, 1, null, 0, 1000),
|
||||
BenchmarkColumnSchema.makeZipf("dimZipf", ValueType.STRING, false, 1, null, 1, 101, 1.0),
|
||||
BenchmarkColumnSchema.makeDiscreteUniform("dimUniform", ValueType.STRING, false, 1, null, 1, 1000000),
|
||||
BenchmarkColumnSchema.makeDiscreteUniform("dimUniform", ValueType.STRING, false, 1, null, 1, 100000),
|
||||
BenchmarkColumnSchema.makeSequential("dimSequentialHalfNull", ValueType.STRING, false, 1, 0.5, 0, 1000),
|
||||
BenchmarkColumnSchema.makeEnumerated(
|
||||
"dimMultivalEnumerated",
|
||||
|
|
|
@ -20,11 +20,11 @@
|
|||
package io.druid.benchmark.query;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.hash.Hashing;
|
||||
import com.google.common.io.Files;
|
||||
|
||||
import io.druid.benchmark.datagen.BenchmarkDataGenerator;
|
||||
import io.druid.benchmark.datagen.BenchmarkSchemaInfo;
|
||||
import io.druid.benchmark.datagen.BenchmarkSchemas;
|
||||
|
@ -44,16 +44,14 @@ import io.druid.query.QueryRunner;
|
|||
import io.druid.query.QueryRunnerFactory;
|
||||
import io.druid.query.QueryToolChest;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.aggregation.AggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleMinAggregatorFactory;
|
||||
import io.druid.query.aggregation.DoubleSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongMaxAggregatorFactory;
|
||||
import io.druid.query.aggregation.LongSumAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesAggregatorFactory;
|
||||
import io.druid.query.aggregation.hyperloglog.HyperUniquesSerde;
|
||||
import io.druid.query.filter.AndDimFilter;
|
||||
import io.druid.query.filter.DimFilter;
|
||||
import io.druid.query.filter.InDimFilter;
|
||||
import io.druid.query.search.SearchQueryQueryToolChest;
|
||||
import io.druid.query.search.SearchQueryRunnerFactory;
|
||||
import io.druid.query.search.SearchResultValue;
|
||||
import io.druid.query.search.SearchStrategySelector;
|
||||
import io.druid.query.search.search.SearchHit;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
|
@ -112,7 +110,6 @@ public class SearchBenchmark
|
|||
private int limit;
|
||||
|
||||
private static final Logger log = new Logger(SearchBenchmark.class);
|
||||
private static final int RNG_SEED = 9999;
|
||||
private static final IndexMergerV9 INDEX_MERGER_V9;
|
||||
private static final IndexIO INDEX_IO;
|
||||
public static final ObjectMapper JSON_MAPPER;
|
||||
|
@ -154,13 +151,6 @@ public class SearchBenchmark
|
|||
{ // basic.A
|
||||
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
|
||||
|
||||
List<AggregatorFactory> queryAggs = new ArrayList<>();
|
||||
queryAggs.add(new LongSumAggregatorFactory("sumLongSequential", "sumLongSequential"));
|
||||
queryAggs.add(new LongMaxAggregatorFactory("maxLongUniform", "maxLongUniform"));
|
||||
queryAggs.add(new DoubleSumAggregatorFactory("sumFloatNormal", "sumFloatNormal"));
|
||||
queryAggs.add(new DoubleMinAggregatorFactory("minFloatZipf", "minFloatZipf"));
|
||||
queryAggs.add(new HyperUniquesAggregatorFactory("hyperUniquesMet", "hyper"));
|
||||
|
||||
Druids.SearchQueryBuilder queryBuilderA =
|
||||
Druids.newSearchQueryBuilder()
|
||||
.dataSource("blah")
|
||||
|
@ -171,6 +161,39 @@ public class SearchBenchmark
|
|||
basicQueries.put("A", queryBuilderA);
|
||||
}
|
||||
|
||||
{ // basic.B
|
||||
QuerySegmentSpec intervalSpec = new MultipleIntervalSegmentSpec(Arrays.asList(basicSchema.getDataInterval()));
|
||||
|
||||
List<String> dimUniformFilterVals = Lists.newArrayList();
|
||||
int resultNum = (int) (100000 * 0.1);
|
||||
int step = 100000 / resultNum;
|
||||
for (int i = 1; i < 100001 && dimUniformFilterVals.size() < resultNum; i += step) {
|
||||
dimUniformFilterVals.add(String.valueOf(i));
|
||||
}
|
||||
|
||||
List<String> dimHyperUniqueFilterVals = Lists.newArrayList();
|
||||
resultNum = (int) (100000 * 0.1);
|
||||
step = 100000 / resultNum;
|
||||
for (int i = 0; i < 100001 && dimHyperUniqueFilterVals.size() < resultNum; i += step) {
|
||||
dimHyperUniqueFilterVals.add(String.valueOf(i));
|
||||
}
|
||||
|
||||
final List<DimFilter> dimFilters = Lists.newArrayList();
|
||||
dimFilters.add(new InDimFilter("dimUniform", dimUniformFilterVals, null));
|
||||
dimFilters.add(new InDimFilter("dimHyperUnique", dimHyperUniqueFilterVals, null));
|
||||
|
||||
Druids.SearchQueryBuilder queryBuilderB =
|
||||
Druids.newSearchQueryBuilder()
|
||||
.dataSource("blah")
|
||||
.granularity(QueryGranularities.ALL)
|
||||
.intervals(intervalSpec)
|
||||
.query("")
|
||||
.dimensions(Lists.newArrayList("dimUniform", "dimHyperUnique"))
|
||||
.filters(new AndDimFilter(dimFilters));
|
||||
|
||||
basicQueries.put("B", queryBuilderB);
|
||||
}
|
||||
|
||||
SCHEMA_QUERY_MAP.put("basic", basicQueries);
|
||||
}
|
||||
|
||||
|
@ -200,7 +223,7 @@ public class SearchBenchmark
|
|||
log.info("Generating rows for segment " + i);
|
||||
BenchmarkDataGenerator gen = new BenchmarkDataGenerator(
|
||||
schemaInfo.getColumnSchemas(),
|
||||
RNG_SEED + i,
|
||||
System.currentTimeMillis(),
|
||||
schemaInfo.getDataInterval(),
|
||||
rowsPerSegment
|
||||
);
|
||||
|
@ -233,9 +256,11 @@ public class SearchBenchmark
|
|||
qIndexes.add(qIndex);
|
||||
}
|
||||
|
||||
final SearchQueryConfig config = new SearchQueryConfig().withOverrides(query);
|
||||
factory = new SearchQueryRunnerFactory(
|
||||
new SearchStrategySelector(Suppliers.ofInstance(config)),
|
||||
new SearchQueryQueryToolChest(
|
||||
new SearchQueryConfig(),
|
||||
config,
|
||||
QueryBenchmarkUtil.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryBenchmarkUtil.NOOP_QUERYWATCHER
|
||||
|
|
|
@ -77,3 +77,43 @@ The format of the result is:
|
|||
}
|
||||
]
|
||||
```
|
||||
|
||||
### Implementation details
|
||||
|
||||
#### Strategies
|
||||
|
||||
Search queries can be executed using two different strategies. The default strategy is determined by the
|
||||
"druid.query.search.searchStrategy" runtime property on the broker. This can be overridden using "searchStrategy" in the
|
||||
query context. If neither the context field nor the property is set, the "useIndexes" strategy will be used.
|
||||
|
||||
- "useIndexes" strategy, the default, first categorizes search dimensions into two groups according to their support for
|
||||
bitmap indexes. And then, it applies index-only and cursor-based execution plans to the group of dimensions supporting
|
||||
bitmaps and others, respectively. The index-only plan uses only indexes for search query processing. For each dimension,
|
||||
it reads the bitmap index for each dimension value, evaluates the search predicate, and finally checks the time interval
|
||||
and filter predicates. For the cursor-based execution plan, please refer to the "cursorOnly" strategy. The index-only
|
||||
plan shows low performance for the search dimensions of large cardinality which means most values of search dimensions
|
||||
are unique.
|
||||
|
||||
- "cursorOnly" strategy generates a cursor-based execution plan. This plan creates a cursor which reads a row from a
|
||||
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:
|
||||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.query.search.searchStrategy`|Default search query strategy.|useIndexes|
|
||||
|
||||
#### Query context
|
||||
|
||||
The following query context parameters apply:
|
||||
|
||||
|Property|Description|
|
||||
|--------|-----------|
|
||||
|`searchStrategy`|Overrides the value of `druid.query.search.searchStrategy` for this query.|
|
||||
|
|
|
@ -20,55 +20,32 @@
|
|||
package io.druid.query.search;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.collections.bitmap.MutableBitmap;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.FunctionalIterable;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.java.util.common.guava.Sequences;
|
||||
import io.druid.query.ColumnSelectorPlus;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.Query;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.dimension.ColumnSelectorStrategy;
|
||||
import io.druid.query.dimension.ColumnSelectorStrategyFactory;
|
||||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.extraction.IdentityExtractionFn;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.search.search.SearchHit;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryExecutor;
|
||||
import io.druid.query.search.search.SearchQuerySpec;
|
||||
import io.druid.segment.ColumnSelectorBitmapIndexSelector;
|
||||
import io.druid.segment.ColumnValueSelector;
|
||||
import io.druid.segment.Cursor;
|
||||
import io.druid.segment.DimensionHandlerUtils;
|
||||
import io.druid.segment.DimensionSelector;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.VirtualColumns;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.GenericColumn;
|
||||
import io.druid.segment.column.ValueType;
|
||||
import io.druid.segment.data.IndexedInts;
|
||||
import io.druid.segment.filter.Filters;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntMap;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntRBTreeMap;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
|
@ -76,12 +53,20 @@ import java.util.Map;
|
|||
*/
|
||||
public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
||||
{
|
||||
private static final SearchStrategyFactory STRATEGY_FACTORY = new SearchStrategyFactory();
|
||||
public static final SearchColumnSelectorStrategyFactory SEARCH_COLUMN_SELECTOR_STRATEGY_FACTORY
|
||||
= new SearchColumnSelectorStrategyFactory();
|
||||
|
||||
private static final EmittingLogger log = new EmittingLogger(SearchQueryRunner.class);
|
||||
private final Segment segment;
|
||||
private final SearchStrategySelector strategySelector;
|
||||
|
||||
private static class SearchStrategyFactory implements ColumnSelectorStrategyFactory<SearchColumnSelectorStrategy>
|
||||
public SearchQueryRunner(Segment segment, SearchStrategySelector strategySelector)
|
||||
{
|
||||
this.segment = segment;
|
||||
this.strategySelector = strategySelector;
|
||||
}
|
||||
|
||||
private static class SearchColumnSelectorStrategyFactory
|
||||
implements ColumnSelectorStrategyFactory<SearchColumnSelectorStrategy>
|
||||
{
|
||||
@Override
|
||||
public SearchColumnSelectorStrategy makeColumnSelectorStrategy(
|
||||
|
@ -89,7 +74,7 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
)
|
||||
{
|
||||
ValueType type = capabilities.getType();
|
||||
switch(type) {
|
||||
switch (type) {
|
||||
case STRING:
|
||||
return new StringSearchColumnSelectorStrategy();
|
||||
default:
|
||||
|
@ -98,11 +83,12 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
}
|
||||
}
|
||||
|
||||
public interface SearchColumnSelectorStrategy<ValueSelectorType extends ColumnValueSelector> extends ColumnSelectorStrategy
|
||||
public interface SearchColumnSelectorStrategy<ValueSelectorType extends ColumnValueSelector>
|
||||
extends ColumnSelectorStrategy
|
||||
{
|
||||
/**
|
||||
* Read the current row from dimSelector and update the search result set.
|
||||
*
|
||||
* <p>
|
||||
* For each row value:
|
||||
* 1. Check if searchQuerySpec accept()s the value
|
||||
* 2. If so, add the value to the result set and increment the counter for that value
|
||||
|
@ -149,11 +135,6 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
}
|
||||
}
|
||||
|
||||
public SearchQueryRunner(Segment segment)
|
||||
{
|
||||
this.segment = segment;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Sequence<Result<SearchResultValue>> run(
|
||||
final Query<Result<SearchResultValue>> input,
|
||||
|
@ -165,81 +146,21 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
}
|
||||
|
||||
final SearchQuery query = (SearchQuery) input;
|
||||
final Filter filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
|
||||
final List<DimensionSpec> dimensions = query.getDimensions();
|
||||
final SearchQuerySpec searchQuerySpec = query.getQuery();
|
||||
final int limit = query.getLimit();
|
||||
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
|
||||
if (intervals.size() != 1) {
|
||||
throw new IAE("Should only have one interval, got[%s]", intervals);
|
||||
}
|
||||
final Interval interval = intervals.get(0);
|
||||
|
||||
// Closing this will cause segfaults in unit tests.
|
||||
final QueryableIndex index = segment.asQueryableIndex();
|
||||
|
||||
final StorageAdapter storageAdapter = segment.asStorageAdapter();
|
||||
|
||||
final List<DimensionSpec> bitmapDims = Lists.newArrayList();
|
||||
final List<DimensionSpec> nonBitmapDims = Lists.newArrayList();
|
||||
partitionDimensionList(index, storageAdapter, dimensions, bitmapDims, nonBitmapDims);
|
||||
|
||||
final Object2IntRBTreeMap<SearchHit> retVal = new Object2IntRBTreeMap<SearchHit>(query.getSort().getComparator());
|
||||
final List<SearchQueryExecutor> plan = strategySelector.strategize(query).getExecutionPlan(query, segment);
|
||||
final Object2IntRBTreeMap<SearchHit> retVal = new Object2IntRBTreeMap<>(query.getSort().getComparator());
|
||||
retVal.defaultReturnValue(0);
|
||||
|
||||
// Get results from bitmap supporting dims first
|
||||
if (!bitmapDims.isEmpty()) {
|
||||
processBitmapDims(index, filter, interval, bitmapDims, searchQuerySpec, limit, retVal);
|
||||
// If there are no non-bitmap dims to search, or we've already hit the result limit, just return now
|
||||
if (nonBitmapDims.size() == 0 || retVal.size() >= limit) {
|
||||
return makeReturnResult(limit, retVal);
|
||||
}
|
||||
int remain = query.getLimit();
|
||||
for (final SearchQueryExecutor executor : plan) {
|
||||
retVal.putAll(executor.execute(remain));
|
||||
remain -= retVal.size();
|
||||
}
|
||||
|
||||
final StorageAdapter adapter = segment.asStorageAdapter();
|
||||
if (adapter == null) {
|
||||
log.makeAlert("WTF!? Unable to process search query on segment.")
|
||||
.addData("segment", segment.getIdentifier())
|
||||
.addData("query", query).emit();
|
||||
throw new ISE(
|
||||
"Null storage adapter found. Probably trying to issue a query against a segment being memory unmapped."
|
||||
);
|
||||
}
|
||||
processNonBitmapDims(query, adapter, filter, interval, limit, nonBitmapDims, searchQuerySpec, retVal);
|
||||
return makeReturnResult(limit, retVal);
|
||||
return makeReturnResult(segment, query.getLimit(), retVal);
|
||||
}
|
||||
|
||||
protected int getStartIndexOfTime(GenericColumn timeValues, long time, boolean inclusive)
|
||||
{
|
||||
int low = 0;
|
||||
int high = timeValues.length() - 1;
|
||||
|
||||
while (low <= high) {
|
||||
int mid = (low + high) >>> 1;
|
||||
long midVal = timeValues.getLongSingleValueRow(mid);
|
||||
|
||||
if (midVal < time) {
|
||||
low = mid + 1;
|
||||
} else if (midVal > time) {
|
||||
high = mid - 1;
|
||||
} else { // key found
|
||||
int i;
|
||||
// rewind the index of the same time values
|
||||
for (i = mid - 1; i >= 0; i--) {
|
||||
long prev = timeValues.getLongSingleValueRow(i);
|
||||
if (time != prev) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return inclusive ? i + 1 : i;
|
||||
}
|
||||
}
|
||||
// key not found.
|
||||
// return insert index
|
||||
return inclusive ? low : low - 1;
|
||||
}
|
||||
|
||||
private Sequence<Result<SearchResultValue>> makeReturnResult(
|
||||
private static Sequence<Result<SearchResultValue>> makeReturnResult(
|
||||
Segment segment,
|
||||
int limit,
|
||||
Object2IntRBTreeMap<SearchHit> retVal
|
||||
)
|
||||
|
@ -258,184 +179,13 @@ public class SearchQueryRunner implements QueryRunner<Result<SearchResultValue>>
|
|||
|
||||
return Sequences.simple(
|
||||
ImmutableList.of(
|
||||
new Result<SearchResultValue>(
|
||||
new Result<>(
|
||||
segment.getDataInterval().getStart(),
|
||||
new SearchResultValue(
|
||||
Lists.newArrayList(new FunctionalIterable<SearchHit>(source).limit(limit))
|
||||
Lists.newArrayList(new FunctionalIterable<>(source).limit(limit))
|
||||
)
|
||||
)
|
||||
)
|
||||
);
|
||||
}
|
||||
|
||||
// Split dimension list into bitmap-supporting list and non-bitmap supporting list
|
||||
private void partitionDimensionList(
|
||||
QueryableIndex index,
|
||||
StorageAdapter storageAdapter,
|
||||
List<DimensionSpec> dimensions,
|
||||
List<DimensionSpec> bitmapDims,
|
||||
List<DimensionSpec> nonBitmapDims
|
||||
)
|
||||
{
|
||||
List<DimensionSpec> dimsToSearch;
|
||||
if (dimensions == null || dimensions.isEmpty()) {
|
||||
dimsToSearch = Lists.newArrayList(Iterables.transform(
|
||||
storageAdapter.getAvailableDimensions(),
|
||||
Druids.DIMENSION_IDENTITY
|
||||
));
|
||||
} else {
|
||||
dimsToSearch = dimensions;
|
||||
}
|
||||
|
||||
if (index != null) {
|
||||
for (DimensionSpec spec : dimsToSearch) {
|
||||
ColumnCapabilities capabilities = storageAdapter.getColumnCapabilities(spec.getDimension());
|
||||
if (capabilities == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (capabilities.hasBitmapIndexes()) {
|
||||
bitmapDims.add(spec);
|
||||
} else {
|
||||
nonBitmapDims.add(spec);
|
||||
}
|
||||
}
|
||||
} else {
|
||||
// no QueryableIndex available, so nothing has bitmaps
|
||||
nonBitmapDims.addAll(dimsToSearch);
|
||||
}
|
||||
}
|
||||
|
||||
private void processNonBitmapDims(
|
||||
SearchQuery query,
|
||||
final StorageAdapter adapter,
|
||||
Filter filter,
|
||||
Interval interval,
|
||||
final int limit,
|
||||
final List<DimensionSpec> nonBitmapDims,
|
||||
final SearchQuerySpec searchQuerySpec,
|
||||
final Object2IntRBTreeMap<SearchHit> retVal
|
||||
)
|
||||
{
|
||||
final Sequence<Cursor> cursors = adapter.makeCursors(
|
||||
filter,
|
||||
interval,
|
||||
VirtualColumns.EMPTY,
|
||||
query.getGranularity(),
|
||||
query.isDescending()
|
||||
);
|
||||
|
||||
cursors.accumulate(
|
||||
retVal,
|
||||
new Accumulator<Object2IntRBTreeMap<SearchHit>, Cursor>()
|
||||
{
|
||||
@Override
|
||||
public Object2IntRBTreeMap<SearchHit> accumulate(Object2IntRBTreeMap<SearchHit> set, Cursor cursor)
|
||||
{
|
||||
if (set.size() >= limit) {
|
||||
return set;
|
||||
}
|
||||
|
||||
List<ColumnSelectorPlus<SearchColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
|
||||
DimensionHandlerUtils.createColumnSelectorPluses(
|
||||
STRATEGY_FACTORY,
|
||||
nonBitmapDims,
|
||||
cursor
|
||||
)
|
||||
);
|
||||
|
||||
while (!cursor.isDone()) {
|
||||
for (ColumnSelectorPlus<SearchColumnSelectorStrategy> selectorPlus : selectorPlusList) {
|
||||
selectorPlus.getColumnSelectorStrategy().updateSearchResultSet(
|
||||
selectorPlus.getOutputName(),
|
||||
selectorPlus.getSelector(),
|
||||
searchQuerySpec,
|
||||
limit,
|
||||
set
|
||||
);
|
||||
|
||||
if (set.size() >= limit) {
|
||||
return set;
|
||||
}
|
||||
}
|
||||
|
||||
cursor.advance();
|
||||
}
|
||||
|
||||
return set;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private void processBitmapDims(
|
||||
QueryableIndex index,
|
||||
Filter filter,
|
||||
Interval interval,
|
||||
List<DimensionSpec> bitmapDims,
|
||||
SearchQuerySpec searchQuerySpec,
|
||||
int limit,
|
||||
final Object2IntRBTreeMap<SearchHit> retVal
|
||||
)
|
||||
{
|
||||
final BitmapFactory bitmapFactory = index.getBitmapFactoryForDimensions();
|
||||
|
||||
final ImmutableBitmap baseFilter =
|
||||
filter == null ? null : filter.getBitmapIndex(new ColumnSelectorBitmapIndexSelector(bitmapFactory, index));
|
||||
|
||||
ImmutableBitmap timeFilteredBitmap;
|
||||
if (!interval.contains(segment.getDataInterval())) {
|
||||
MutableBitmap timeBitmap = bitmapFactory.makeEmptyMutableBitmap();
|
||||
final Column timeColumn = index.getColumn(Column.TIME_COLUMN_NAME);
|
||||
try (final GenericColumn timeValues = timeColumn.getGenericColumn()) {
|
||||
|
||||
int startIndex = Math.max(0, getStartIndexOfTime(timeValues, interval.getStartMillis(), true));
|
||||
int endIndex = Math.min(
|
||||
timeValues.length() - 1,
|
||||
getStartIndexOfTime(timeValues, interval.getEndMillis(), false)
|
||||
);
|
||||
|
||||
for (int i = startIndex; i <= endIndex; i++) {
|
||||
timeBitmap.add(i);
|
||||
}
|
||||
|
||||
final ImmutableBitmap finalTimeBitmap = bitmapFactory.makeImmutableBitmap(timeBitmap);
|
||||
timeFilteredBitmap =
|
||||
(baseFilter == null) ? finalTimeBitmap : finalTimeBitmap.intersection(baseFilter);
|
||||
}
|
||||
} else {
|
||||
timeFilteredBitmap = baseFilter;
|
||||
}
|
||||
|
||||
for (DimensionSpec dimension : bitmapDims) {
|
||||
final Column column = index.getColumn(dimension.getDimension());
|
||||
if (column == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final BitmapIndex bitmapIndex = column.getBitmapIndex();
|
||||
ExtractionFn extractionFn = dimension.getExtractionFn();
|
||||
if (extractionFn == null) {
|
||||
extractionFn = IdentityExtractionFn.getInstance();
|
||||
}
|
||||
if (bitmapIndex != null) {
|
||||
for (int i = 0; i < bitmapIndex.getCardinality(); ++i) {
|
||||
String dimVal = Strings.nullToEmpty(extractionFn.apply(bitmapIndex.getValue(i)));
|
||||
if (!searchQuerySpec.accept(dimVal)) {
|
||||
continue;
|
||||
}
|
||||
ImmutableBitmap bitmap = bitmapIndex.getBitmap(i);
|
||||
if (timeFilteredBitmap != null) {
|
||||
bitmap = bitmapFactory.intersection(Arrays.asList(timeFilteredBitmap, bitmap));
|
||||
}
|
||||
if (bitmap.size() > 0) {
|
||||
retVal.addTo(new SearchHit(dimension.getOutputName(), dimVal), bitmap.size());
|
||||
if (retVal.size() >= limit) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,15 +35,18 @@ import java.util.concurrent.ExecutorService;
|
|||
*/
|
||||
public class SearchQueryRunnerFactory implements QueryRunnerFactory<Result<SearchResultValue>, SearchQuery>
|
||||
{
|
||||
private final SearchStrategySelector strategySelector;
|
||||
private final SearchQueryQueryToolChest toolChest;
|
||||
private final QueryWatcher queryWatcher;
|
||||
|
||||
@Inject
|
||||
public SearchQueryRunnerFactory(
|
||||
SearchStrategySelector strategySelector,
|
||||
SearchQueryQueryToolChest toolChest,
|
||||
QueryWatcher queryWatcher
|
||||
)
|
||||
{
|
||||
this.strategySelector = strategySelector;
|
||||
this.toolChest = toolChest;
|
||||
this.queryWatcher = queryWatcher;
|
||||
}
|
||||
|
@ -51,7 +54,7 @@ public class SearchQueryRunnerFactory implements QueryRunnerFactory<Result<Searc
|
|||
@Override
|
||||
public QueryRunner<Result<SearchResultValue>> createRunner(final Segment segment)
|
||||
{
|
||||
return new SearchQueryRunner(segment);
|
||||
return new SearchQueryRunner(segment, strategySelector);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -0,0 +1,62 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.inject.Inject;
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.java.util.common.ISE;
|
||||
import io.druid.query.search.search.AutoStrategy;
|
||||
import io.druid.query.search.search.CursorOnlyStrategy;
|
||||
import io.druid.query.search.search.UseIndexesStrategy;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
import io.druid.query.search.search.SearchStrategy;
|
||||
|
||||
public class SearchStrategySelector
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(SearchStrategySelector.class);
|
||||
private final SearchQueryConfig config;
|
||||
|
||||
@Inject
|
||||
public SearchStrategySelector(Supplier<SearchQueryConfig> configSupplier)
|
||||
{
|
||||
this.config = configSupplier.get();
|
||||
}
|
||||
|
||||
public SearchStrategy strategize(SearchQuery query)
|
||||
{
|
||||
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 UseIndexesStrategy.NAME:
|
||||
log.debug("Use-index strategy is selected, query id [%s]", query.getId());
|
||||
return UseIndexesStrategy.of(query);
|
||||
case CursorOnlyStrategy.NAME:
|
||||
log.debug("Cursor-only strategy is selected, query id [%s]", query.getId());
|
||||
return CursorOnlyStrategy.of(query);
|
||||
default:
|
||||
throw new ISE("Unknown strategy[%s], query id [%s]", strategyString, query.getId());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,128 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
import com.metamx.emitter.EmittingLogger;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.segment.ColumnSelectorBitmapIndexSelector;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.column.Column;
|
||||
|
||||
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 BitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
|
||||
index.getBitmapFactoryForDimensions(),
|
||||
index
|
||||
);
|
||||
|
||||
// Index-only plan is used only when any filter is not specified or every 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 filter, and then use it to compute the filtered result by intersecting bitmaps.
|
||||
if (filter == null || filter.supportsBitmapIndex(selector)) {
|
||||
final ImmutableBitmap timeFilteredBitmap = UseIndexesStrategy.makeTimeFilteredBitmap(
|
||||
index,
|
||||
segment,
|
||||
filter,
|
||||
interval
|
||||
);
|
||||
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 =
|
||||
(timeFilteredBitmap == null ? index.getNumRows() : timeFilteredBitmap.size()) * 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.withTimeFilteredBitmap(query, timeFilteredBitmap).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 Column column = index.getColumn(dimension.getDimension());
|
||||
if (column != null) {
|
||||
final BitmapIndex bitmapIndex = column.getBitmapIndex();
|
||||
if (bitmapIndex != null) {
|
||||
totalCard += bitmapIndex.getCardinality();
|
||||
}
|
||||
}
|
||||
}
|
||||
return totalCard;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
public class ConciseBitmapDecisionHelper extends SearchQueryDecisionHelper
|
||||
{
|
||||
// This value comes from an experiment.
|
||||
// See the discussion at https://github.com/druid-io/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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,149 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.java.util.common.guava.Accumulator;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
import io.druid.query.ColumnSelectorPlus;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.search.SearchQueryRunner;
|
||||
import io.druid.query.search.SearchQueryRunner.SearchColumnSelectorStrategy;
|
||||
import io.druid.segment.Cursor;
|
||||
import io.druid.segment.DimensionHandlerUtils;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.VirtualColumns;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntRBTreeMap;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public class CursorOnlyStrategy extends SearchStrategy
|
||||
{
|
||||
public static final String NAME = "cursorOnly";
|
||||
|
||||
public static CursorOnlyStrategy of(SearchQuery query)
|
||||
{
|
||||
return new CursorOnlyStrategy(query);
|
||||
}
|
||||
|
||||
private CursorOnlyStrategy(SearchQuery query)
|
||||
{
|
||||
super(query);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<SearchQueryExecutor> getExecutionPlan(SearchQuery query, Segment segment)
|
||||
{
|
||||
final StorageAdapter adapter = segment.asStorageAdapter();
|
||||
final List<DimensionSpec> dimensionSpecs = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions());
|
||||
return ImmutableList.<SearchQueryExecutor>of(new CursorBasedExecutor(
|
||||
query,
|
||||
segment,
|
||||
filter,
|
||||
interval,
|
||||
dimensionSpecs
|
||||
));
|
||||
}
|
||||
|
||||
public static class CursorBasedExecutor extends SearchQueryExecutor
|
||||
{
|
||||
|
||||
protected Filter filter;
|
||||
protected Interval interval;
|
||||
|
||||
public CursorBasedExecutor(
|
||||
SearchQuery query,
|
||||
Segment segment,
|
||||
Filter filter,
|
||||
Interval interval, List<DimensionSpec> dimensionSpecs
|
||||
)
|
||||
{
|
||||
super(query, segment, dimensionSpecs);
|
||||
|
||||
this.filter = filter;
|
||||
this.interval = interval;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object2IntRBTreeMap<SearchHit> execute(final int limit)
|
||||
{
|
||||
final StorageAdapter adapter = segment.asStorageAdapter();
|
||||
|
||||
final Sequence<Cursor> cursors = adapter.makeCursors(
|
||||
filter,
|
||||
interval,
|
||||
VirtualColumns.EMPTY,
|
||||
query.getGranularity(),
|
||||
query.isDescending()
|
||||
);
|
||||
|
||||
final Object2IntRBTreeMap<SearchHit> retVal = new Object2IntRBTreeMap<>(query.getSort().getComparator());
|
||||
retVal.defaultReturnValue(0);
|
||||
|
||||
cursors.accumulate(
|
||||
retVal,
|
||||
new Accumulator<Object2IntRBTreeMap<SearchHit>, Cursor>()
|
||||
{
|
||||
@Override
|
||||
public Object2IntRBTreeMap<SearchHit> accumulate(Object2IntRBTreeMap<SearchHit> set, Cursor cursor)
|
||||
{
|
||||
if (set.size() >= limit) {
|
||||
return set;
|
||||
}
|
||||
|
||||
final List<ColumnSelectorPlus<SearchColumnSelectorStrategy>> selectorPlusList = Arrays.asList(
|
||||
DimensionHandlerUtils.createColumnSelectorPluses(
|
||||
SearchQueryRunner.SEARCH_COLUMN_SELECTOR_STRATEGY_FACTORY,
|
||||
dimsToSearch,
|
||||
cursor
|
||||
)
|
||||
);
|
||||
|
||||
while (!cursor.isDone()) {
|
||||
for (ColumnSelectorPlus<SearchColumnSelectorStrategy> selectorPlus : selectorPlusList) {
|
||||
selectorPlus.getColumnSelectorStrategy().updateSearchResultSet(
|
||||
selectorPlus.getOutputName(),
|
||||
selectorPlus.getSelector(),
|
||||
searchQuerySpec,
|
||||
limit,
|
||||
set
|
||||
);
|
||||
|
||||
if (set.size() >= limit) {
|
||||
return set;
|
||||
}
|
||||
}
|
||||
|
||||
cursor.advance();
|
||||
}
|
||||
|
||||
return set;
|
||||
}
|
||||
}
|
||||
);
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,38 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
public class RoaringBitmapDecisionHelper extends SearchQueryDecisionHelper
|
||||
{
|
||||
// This value comes from an experiment.
|
||||
// See the discussion at https://github.com/druid-io/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);
|
||||
}
|
||||
}
|
|
@ -64,14 +64,14 @@ public class SearchQuery extends BaseQuery<Result<SearchResultValue>>
|
|||
)
|
||||
{
|
||||
super(dataSource, querySegmentSpec, false, context);
|
||||
Preconditions.checkNotNull(querySegmentSpec, "Must specify an interval");
|
||||
|
||||
this.dimFilter = dimFilter;
|
||||
this.sortSpec = sortSpec == null ? DEFAULT_SORT_SPEC : sortSpec;
|
||||
this.granularity = granularity == null ? QueryGranularities.ALL : granularity;
|
||||
this.limit = (limit == 0) ? 1000 : limit;
|
||||
this.dimensions = dimensions;
|
||||
this.querySpec = querySpec == null ? new AllSearchQuerySpec() : querySpec;
|
||||
|
||||
Preconditions.checkNotNull(querySegmentSpec, "Must specify an interval");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,12 +27,35 @@ import javax.validation.constraints.Min;
|
|||
*/
|
||||
public class SearchQueryConfig
|
||||
{
|
||||
public static final String CTX_KEY_STRATEGY = "searchStrategy";
|
||||
|
||||
@JsonProperty
|
||||
@Min(1)
|
||||
private int maxSearchLimit = 1000;
|
||||
|
||||
@JsonProperty
|
||||
private String searchStrategy = UseIndexesStrategy.NAME;
|
||||
|
||||
public int getMaxSearchLimit()
|
||||
{
|
||||
return maxSearchLimit;
|
||||
}
|
||||
|
||||
public String getSearchStrategy()
|
||||
{
|
||||
return searchStrategy;
|
||||
}
|
||||
|
||||
public void setSearchStrategy(final String strategy)
|
||||
{
|
||||
this.searchStrategy = strategy;
|
||||
}
|
||||
|
||||
public SearchQueryConfig withOverrides(final SearchQuery query)
|
||||
{
|
||||
final SearchQueryConfig newConfig = new SearchQueryConfig();
|
||||
newConfig.maxSearchLimit = query.getLimit();
|
||||
newConfig.searchStrategy = query.getContextValue(CTX_KEY_STRATEGY, searchStrategy);
|
||||
return newConfig;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,35 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
public abstract class SearchQueryDecisionHelper
|
||||
{
|
||||
private final double bitmapIntersectCost;
|
||||
|
||||
protected SearchQueryDecisionHelper(final double bitmapIntersectCost)
|
||||
{
|
||||
this.bitmapIntersectCost = bitmapIntersectCost;
|
||||
}
|
||||
|
||||
public double getBitmapIntersectCost()
|
||||
{
|
||||
return bitmapIntersectCost;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,44 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.segment.Segment;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntRBTreeMap;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public abstract class SearchQueryExecutor
|
||||
{
|
||||
protected final SearchQuery query;
|
||||
protected final SearchQuerySpec searchQuerySpec;
|
||||
protected final Segment segment;
|
||||
protected final List<DimensionSpec> dimsToSearch;
|
||||
|
||||
public SearchQueryExecutor(SearchQuery query, Segment segment, List<DimensionSpec> dimensionSpecs)
|
||||
{
|
||||
this.query = query;
|
||||
this.segment = segment;
|
||||
this.searchQuerySpec = query.getQuery();
|
||||
this.dimsToSearch = dimensionSpecs;
|
||||
}
|
||||
|
||||
public abstract Object2IntRBTreeMap<SearchHit> execute(int limit);
|
||||
}
|
|
@ -0,0 +1,76 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.collections.bitmap.ConciseBitmapFactory;
|
||||
import io.druid.collections.bitmap.RoaringBitmapFactory;
|
||||
import io.druid.java.util.common.IAE;
|
||||
import io.druid.query.Druids;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.data.Indexed;
|
||||
import io.druid.segment.filter.Filters;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public abstract class SearchStrategy
|
||||
{
|
||||
protected final Filter filter;
|
||||
protected final Interval interval;
|
||||
|
||||
protected SearchStrategy(SearchQuery query)
|
||||
{
|
||||
this.filter = Filters.convertToCNFFromQueryContext(query, Filters.toFilter(query.getDimensionsFilter()));
|
||||
final List<Interval> intervals = query.getQuerySegmentSpec().getIntervals();
|
||||
if (intervals.size() != 1) {
|
||||
throw new IAE("Should only have one interval, got[%s]", intervals);
|
||||
}
|
||||
this.interval = intervals.get(0);
|
||||
}
|
||||
|
||||
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().getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
static List<DimensionSpec> getDimsToSearch(Indexed<String> availableDimensions, List<DimensionSpec> dimensions)
|
||||
{
|
||||
if (dimensions == null || dimensions.isEmpty()) {
|
||||
return ImmutableList.copyOf(Iterables.transform(availableDimensions, Druids.DIMENSION_IDENTITY));
|
||||
} else {
|
||||
return dimensions;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,299 @@
|
|||
/*
|
||||
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. Metamarkets 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 io.druid.query.search.search;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.collections.bitmap.BitmapFactory;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.collections.bitmap.MutableBitmap;
|
||||
import io.druid.java.util.common.Pair;
|
||||
import io.druid.query.dimension.DimensionSpec;
|
||||
import io.druid.query.extraction.ExtractionFn;
|
||||
import io.druid.query.extraction.IdentityExtractionFn;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
import io.druid.query.filter.Filter;
|
||||
import io.druid.query.search.search.CursorOnlyStrategy.CursorBasedExecutor;
|
||||
import io.druid.segment.ColumnSelectorBitmapIndexSelector;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.Segment;
|
||||
import io.druid.segment.StorageAdapter;
|
||||
import io.druid.segment.column.BitmapIndex;
|
||||
import io.druid.segment.column.Column;
|
||||
import io.druid.segment.column.ColumnCapabilities;
|
||||
import io.druid.segment.column.GenericColumn;
|
||||
import it.unimi.dsi.fastutil.objects.Object2IntRBTreeMap;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
public class UseIndexesStrategy extends SearchStrategy
|
||||
{
|
||||
public static final String NAME = "useIndexes";
|
||||
|
||||
private final ImmutableBitmap timeFilteredBitmap;
|
||||
private final boolean needToMakeFilteredBitmap;
|
||||
|
||||
public static UseIndexesStrategy of(SearchQuery query)
|
||||
{
|
||||
return new UseIndexesStrategy(query, true, null);
|
||||
}
|
||||
|
||||
public static UseIndexesStrategy withTimeFilteredBitmap(
|
||||
SearchQuery query,
|
||||
@Nullable ImmutableBitmap timeFilteredBitmap
|
||||
)
|
||||
{
|
||||
return new UseIndexesStrategy(query, false, timeFilteredBitmap);
|
||||
}
|
||||
|
||||
private UseIndexesStrategy(
|
||||
SearchQuery query,
|
||||
boolean needToMakeFilteredBitmap,
|
||||
@Nullable ImmutableBitmap timeFilteredBitmap
|
||||
)
|
||||
{
|
||||
super(query);
|
||||
this.needToMakeFilteredBitmap = needToMakeFilteredBitmap;
|
||||
this.timeFilteredBitmap = timeFilteredBitmap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<SearchQueryExecutor> getExecutionPlan(SearchQuery query, Segment segment)
|
||||
{
|
||||
final ImmutableList.Builder<SearchQueryExecutor> builder = ImmutableList.builder();
|
||||
final QueryableIndex index = segment.asQueryableIndex();
|
||||
final StorageAdapter adapter = segment.asStorageAdapter();
|
||||
final List<DimensionSpec> searchDims = getDimsToSearch(adapter.getAvailableDimensions(), query.getDimensions());
|
||||
|
||||
if (index != null) {
|
||||
final Pair<List<DimensionSpec>, List<DimensionSpec>> pair = // pair of bitmap dims and non-bitmap dims
|
||||
partitionDimensionList(adapter, searchDims);
|
||||
final List<DimensionSpec> bitmapSuppDims = pair.lhs;
|
||||
final List<DimensionSpec> nonBitmapSuppDims = pair.rhs;
|
||||
|
||||
if (bitmapSuppDims.size() > 0) {
|
||||
final BitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
|
||||
index.getBitmapFactoryForDimensions(),
|
||||
index
|
||||
);
|
||||
|
||||
// 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 filter, and then use it to compute the filtered result by intersecting bitmaps.
|
||||
if (filter == null || filter.supportsBitmapIndex(selector)) {
|
||||
final ImmutableBitmap timeFilteredBitmap = this.needToMakeFilteredBitmap ?
|
||||
makeTimeFilteredBitmap(index, segment, filter, interval) :
|
||||
this.timeFilteredBitmap;
|
||||
builder.add(new IndexOnlyExecutor(query, segment, timeFilteredBitmap, bitmapSuppDims));
|
||||
} else {
|
||||
// Fall back to cursor-based execution strategy
|
||||
nonBitmapSuppDims.addAll(bitmapSuppDims);
|
||||
}
|
||||
}
|
||||
|
||||
if (nonBitmapSuppDims.size() > 0) {
|
||||
builder.add(new CursorBasedExecutor(query, segment, filter, interval, nonBitmapSuppDims));
|
||||
}
|
||||
} else {
|
||||
builder.add(new CursorBasedExecutor(query, segment, filter, interval, searchDims));
|
||||
}
|
||||
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
// Split dimension list into bitmap-supporting list and non-bitmap supporting list
|
||||
private static Pair<List<DimensionSpec>, List<DimensionSpec>> partitionDimensionList(
|
||||
StorageAdapter adapter,
|
||||
List<DimensionSpec> dimensions
|
||||
)
|
||||
{
|
||||
final List<DimensionSpec> bitmapDims = Lists.newArrayList();
|
||||
final List<DimensionSpec> nonBitmapDims = Lists.newArrayList();
|
||||
final List<DimensionSpec> dimsToSearch = getDimsToSearch(
|
||||
adapter.getAvailableDimensions(),
|
||||
dimensions
|
||||
);
|
||||
|
||||
for (DimensionSpec spec : dimsToSearch) {
|
||||
ColumnCapabilities capabilities = adapter.getColumnCapabilities(spec.getDimension());
|
||||
if (capabilities == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
if (capabilities.hasBitmapIndexes()) {
|
||||
bitmapDims.add(spec);
|
||||
} else {
|
||||
nonBitmapDims.add(spec);
|
||||
}
|
||||
}
|
||||
|
||||
return new Pair<List<DimensionSpec>, List<DimensionSpec>>(
|
||||
ImmutableList.copyOf(bitmapDims),
|
||||
ImmutableList.copyOf(nonBitmapDims)
|
||||
);
|
||||
}
|
||||
|
||||
static ImmutableBitmap makeTimeFilteredBitmap(
|
||||
final QueryableIndex index,
|
||||
final Segment segment,
|
||||
final Filter filter,
|
||||
final Interval interval
|
||||
)
|
||||
{
|
||||
final BitmapFactory bitmapFactory = index.getBitmapFactoryForDimensions();
|
||||
final ImmutableBitmap baseFilter;
|
||||
if (filter == null) {
|
||||
baseFilter = null;
|
||||
} else {
|
||||
final BitmapIndexSelector selector = new ColumnSelectorBitmapIndexSelector(
|
||||
index.getBitmapFactoryForDimensions(),
|
||||
index
|
||||
);
|
||||
Preconditions.checkArgument(filter.supportsBitmapIndex(selector), "filter[%s] should support bitmap", filter);
|
||||
baseFilter = filter.getBitmapIndex(selector);
|
||||
}
|
||||
|
||||
final ImmutableBitmap timeFilteredBitmap;
|
||||
if (!interval.contains(segment.getDataInterval())) {
|
||||
final MutableBitmap timeBitmap = bitmapFactory.makeEmptyMutableBitmap();
|
||||
final Column timeColumn = index.getColumn(Column.TIME_COLUMN_NAME);
|
||||
try (final GenericColumn timeValues = timeColumn.getGenericColumn()) {
|
||||
|
||||
int startIndex = Math.max(0, getStartIndexOfTime(timeValues, interval.getStartMillis(), true));
|
||||
int endIndex = Math.min(
|
||||
timeValues.length() - 1,
|
||||
getStartIndexOfTime(timeValues, interval.getEndMillis(), false)
|
||||
);
|
||||
|
||||
for (int i = startIndex; i <= endIndex; i++) {
|
||||
timeBitmap.add(i);
|
||||
}
|
||||
|
||||
final ImmutableBitmap finalTimeBitmap = bitmapFactory.makeImmutableBitmap(timeBitmap);
|
||||
timeFilteredBitmap =
|
||||
(baseFilter == null) ? finalTimeBitmap : finalTimeBitmap.intersection(baseFilter);
|
||||
}
|
||||
} else {
|
||||
timeFilteredBitmap = baseFilter;
|
||||
}
|
||||
|
||||
return timeFilteredBitmap;
|
||||
}
|
||||
|
||||
private static int getStartIndexOfTime(GenericColumn timeValues, long time, boolean inclusive)
|
||||
{
|
||||
int low = 0;
|
||||
int high = timeValues.length() - 1;
|
||||
|
||||
while (low <= high) {
|
||||
int mid = (low + high) >>> 1;
|
||||
long midVal = timeValues.getLongSingleValueRow(mid);
|
||||
|
||||
if (midVal < time) {
|
||||
low = mid + 1;
|
||||
} else if (midVal > time) {
|
||||
high = mid - 1;
|
||||
} else { // key found
|
||||
int i;
|
||||
// rewind the index of the same time values
|
||||
for (i = mid - 1; i >= 0; i--) {
|
||||
long prev = timeValues.getLongSingleValueRow(i);
|
||||
if (time != prev) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
return inclusive ? i + 1 : i;
|
||||
}
|
||||
}
|
||||
// key not found.
|
||||
// return insert index
|
||||
return inclusive ? low : low - 1;
|
||||
}
|
||||
|
||||
public static class IndexOnlyExecutor extends SearchQueryExecutor
|
||||
{
|
||||
|
||||
private final ImmutableBitmap timeFilteredBitmap;
|
||||
|
||||
public IndexOnlyExecutor(
|
||||
SearchQuery query, Segment segment,
|
||||
ImmutableBitmap timeFilteredBitmap,
|
||||
List<DimensionSpec> dimensionSpecs
|
||||
)
|
||||
{
|
||||
super(query, segment, dimensionSpecs);
|
||||
this.timeFilteredBitmap = timeFilteredBitmap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Object2IntRBTreeMap<SearchHit> execute(int limit)
|
||||
{
|
||||
final QueryableIndex index = segment.asQueryableIndex();
|
||||
Preconditions.checkArgument(index != null, "Index should not be null");
|
||||
|
||||
final Object2IntRBTreeMap<SearchHit> retVal = new Object2IntRBTreeMap<>(query.getSort().getComparator());
|
||||
retVal.defaultReturnValue(0);
|
||||
|
||||
final BitmapFactory bitmapFactory = index.getBitmapFactoryForDimensions();
|
||||
|
||||
for (DimensionSpec dimension : dimsToSearch) {
|
||||
final Column column = index.getColumn(dimension.getDimension());
|
||||
if (column == null) {
|
||||
continue;
|
||||
}
|
||||
|
||||
final BitmapIndex bitmapIndex = column.getBitmapIndex();
|
||||
Preconditions.checkArgument(bitmapIndex != null,
|
||||
"Dimension [%s] should support bitmap index", dimension.getDimension()
|
||||
);
|
||||
|
||||
ExtractionFn extractionFn = dimension.getExtractionFn();
|
||||
if (extractionFn == null) {
|
||||
extractionFn = IdentityExtractionFn.getInstance();
|
||||
}
|
||||
for (int i = 0; i < bitmapIndex.getCardinality(); ++i) {
|
||||
String dimVal = Strings.nullToEmpty(extractionFn.apply(bitmapIndex.getValue(i)));
|
||||
if (!searchQuerySpec.accept(dimVal)) {
|
||||
continue;
|
||||
}
|
||||
ImmutableBitmap bitmap = bitmapIndex.getBitmap(i);
|
||||
if (timeFilteredBitmap != null) {
|
||||
bitmap = bitmapFactory.intersection(Arrays.asList(timeFilteredBitmap, bitmap));
|
||||
}
|
||||
if (bitmap.size() > 0) {
|
||||
retVal.addTo(new SearchHit(dimension.getOutputName(), dimVal), bitmap.size());
|
||||
if (retVal.size() >= limit) {
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return retVal;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.segment.filter;
|
||||
|
||||
import com.google.common.base.Joiner;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.collections.bitmap.ImmutableBitmap;
|
||||
import io.druid.query.filter.BitmapIndexSelector;
|
||||
|
@ -55,6 +56,9 @@ public class AndFilter implements BooleanFilter
|
|||
|
||||
final List<ImmutableBitmap> bitmaps = Lists.newArrayListWithCapacity(filters.size());
|
||||
for (final Filter filter : filters) {
|
||||
Preconditions.checkArgument(filter.supportsBitmapIndex(selector),
|
||||
"Filter[%s] does not support bitmap index", filter
|
||||
);
|
||||
final ImmutableBitmap bitmapIndex = filter.getBitmapIndex(selector);
|
||||
if (bitmapIndex.isEmpty()) {
|
||||
// Short-circuit.
|
||||
|
|
|
@ -20,9 +20,11 @@
|
|||
package io.druid.query;
|
||||
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import io.druid.collections.StupidPool;
|
||||
import io.druid.query.search.SearchQueryQueryToolChest;
|
||||
import io.druid.query.search.SearchQueryRunnerFactory;
|
||||
import io.druid.query.search.SearchStrategySelector;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
import io.druid.query.timeboundary.TimeBoundaryQueryRunnerFactory;
|
||||
import io.druid.query.timeseries.TimeseriesQueryEngine;
|
||||
|
@ -63,8 +65,10 @@ public class TestQueryRunners
|
|||
{
|
||||
QueryRunnerFactory factory = new TopNQueryRunnerFactory(
|
||||
pool,
|
||||
new TopNQueryQueryToolChest(topNConfig,
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
|
||||
new TopNQueryQueryToolChest(
|
||||
topNConfig,
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
);
|
||||
return new FinalizeResultsQueryRunner<T>(
|
||||
|
@ -94,10 +98,15 @@ public class TestQueryRunners
|
|||
Segment adapter
|
||||
)
|
||||
{
|
||||
QueryRunnerFactory factory = new SearchQueryRunnerFactory(new SearchQueryQueryToolChest(
|
||||
new SearchQueryConfig(),
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER);
|
||||
final SearchQueryConfig config = new SearchQueryConfig();
|
||||
QueryRunnerFactory factory = new SearchQueryRunnerFactory(
|
||||
new SearchStrategySelector(Suppliers.ofInstance(config)),
|
||||
new SearchQueryQueryToolChest(
|
||||
config,
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
);
|
||||
return new FinalizeResultsQueryRunner<T>(
|
||||
factory.createRunner(adapter),
|
||||
factory.getToolchest()
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.query.search;
|
||||
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Lists;
|
||||
import io.druid.java.util.common.guava.Sequence;
|
||||
|
@ -64,17 +65,20 @@ import java.util.Map;
|
|||
public class SearchQueryRunnerTest
|
||||
{
|
||||
private static final Logger LOG = new Logger(SearchQueryRunnerTest.class);
|
||||
private static final SearchQueryConfig config = new SearchQueryConfig();
|
||||
private static final SearchQueryQueryToolChest toolChest = new SearchQueryQueryToolChest(
|
||||
new SearchQueryConfig(),
|
||||
config,
|
||||
QueryRunnerTestHelper.NoopIntervalChunkingQueryRunnerDecorator()
|
||||
);
|
||||
private static final SearchStrategySelector selector = new SearchStrategySelector(Suppliers.ofInstance(config));
|
||||
|
||||
@Parameterized.Parameters(name="{0}")
|
||||
@Parameterized.Parameters(name = "{0}")
|
||||
public static Iterable<Object[]> constructorFeeder() throws IOException
|
||||
{
|
||||
return QueryRunnerTestHelper.transformToConstructionFeeder(
|
||||
QueryRunnerTestHelper.makeQueryRunners(
|
||||
new SearchQueryRunnerFactory(
|
||||
selector,
|
||||
toolChest,
|
||||
QueryRunnerTestHelper.NOOP_QUERYWATCHER
|
||||
)
|
||||
|
@ -389,7 +393,8 @@ public class SearchQueryRunnerTest
|
|||
new AndDimFilter(
|
||||
Arrays.<DimFilter>asList(
|
||||
new SelectorDimFilter(QueryRunnerTestHelper.marketDimension, "total_market", null),
|
||||
new SelectorDimFilter(QueryRunnerTestHelper.qualityDimension, "mezzanine", null))))
|
||||
new SelectorDimFilter(QueryRunnerTestHelper.qualityDimension, "mezzanine", null)
|
||||
)))
|
||||
.intervals(QueryRunnerTestHelper.fullOnInterval)
|
||||
.dimensions(QueryRunnerTestHelper.qualityDimension)
|
||||
.query("a")
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
|
||||
package io.druid.query.search;
|
||||
|
||||
import com.google.common.base.Suppliers;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
|
@ -27,9 +28,12 @@ import io.druid.java.util.common.guava.Sequences;
|
|||
import io.druid.query.Druids;
|
||||
import io.druid.query.QueryRunner;
|
||||
import io.druid.query.Result;
|
||||
import io.druid.query.search.search.AutoStrategy;
|
||||
import io.druid.query.search.search.CursorOnlyStrategy;
|
||||
import io.druid.query.search.search.SearchHit;
|
||||
import io.druid.query.search.search.SearchQuery;
|
||||
import io.druid.query.search.search.SearchQueryConfig;
|
||||
import io.druid.query.search.search.UseIndexesStrategy;
|
||||
import io.druid.segment.IncrementalIndexSegment;
|
||||
import io.druid.segment.QueryableIndex;
|
||||
import io.druid.segment.QueryableIndexSegment;
|
||||
|
@ -68,13 +72,13 @@ public class SearchQueryRunnerWithCaseTest
|
|||
@Parameterized.Parameters
|
||||
public static Iterable<Object[]> constructorFeeder() throws IOException
|
||||
{
|
||||
SearchQueryRunnerFactory factory = new SearchQueryRunnerFactory(
|
||||
new SearchQueryQueryToolChest(
|
||||
new SearchQueryConfig(),
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
NOOP_QUERYWATCHER
|
||||
);
|
||||
final SearchQueryConfig[] configs = new SearchQueryConfig[3];
|
||||
configs[0] = new SearchQueryConfig();
|
||||
configs[0].setSearchStrategy(UseIndexesStrategy.NAME);
|
||||
configs[1] = new SearchQueryConfig();
|
||||
configs[1].setSearchStrategy(CursorOnlyStrategy.NAME);
|
||||
configs[2] = new SearchQueryConfig();
|
||||
configs[2].setSearchStrategy(AutoStrategy.NAME);
|
||||
|
||||
CharSource input = CharSource.wrap(
|
||||
"2011-01-12T00:00:00.000Z\tspot\tAutoMotive\tPREFERRED\ta\u0001preferred\t100.000000\n" +
|
||||
|
@ -89,13 +93,48 @@ public class SearchQueryRunnerWithCaseTest
|
|||
QueryableIndex index3 = TestIndex.persistRealtimeAndLoadMMapped(index1);
|
||||
QueryableIndex index4 = TestIndex.persistRealtimeAndLoadMMapped(index2);
|
||||
|
||||
return transformToConstructionFeeder(
|
||||
Arrays.asList(
|
||||
makeQueryRunner(factory, "index1", new IncrementalIndexSegment(index1, "index1"), "index1"),
|
||||
makeQueryRunner(factory, "index2", new IncrementalIndexSegment(index2, "index2"), "index2"),
|
||||
makeQueryRunner(factory, "index3", new QueryableIndexSegment("index3", index3), "index3"),
|
||||
makeQueryRunner(factory, "index4", new QueryableIndexSegment("index4", index4), "index4")
|
||||
final List<QueryRunner<Result<SearchResultValue>>> runners = Lists.newArrayList();
|
||||
for (int i = 0; i < configs.length; i++) {
|
||||
runners.addAll(Arrays.asList(
|
||||
makeQueryRunner(
|
||||
makeRunnerFactory(configs[i]),
|
||||
"index1",
|
||||
new IncrementalIndexSegment(index1, "index1"),
|
||||
"index1"
|
||||
),
|
||||
makeQueryRunner(
|
||||
makeRunnerFactory(configs[i]),
|
||||
"index2",
|
||||
new IncrementalIndexSegment(index2, "index2"),
|
||||
"index2"
|
||||
),
|
||||
makeQueryRunner(
|
||||
makeRunnerFactory(configs[i]),
|
||||
"index3",
|
||||
new QueryableIndexSegment("index3", index3),
|
||||
"index3"
|
||||
),
|
||||
makeQueryRunner(
|
||||
makeRunnerFactory(configs[i]),
|
||||
"index4",
|
||||
new QueryableIndexSegment("index4", index4),
|
||||
"index4"
|
||||
)
|
||||
));
|
||||
}
|
||||
|
||||
return transformToConstructionFeeder(runners);
|
||||
}
|
||||
|
||||
static SearchQueryRunnerFactory makeRunnerFactory(final SearchQueryConfig config)
|
||||
{
|
||||
return new SearchQueryRunnerFactory(
|
||||
new SearchStrategySelector(Suppliers.ofInstance(config)),
|
||||
new SearchQueryQueryToolChest(
|
||||
config,
|
||||
NoopIntervalChunkingQueryRunnerDecorator()
|
||||
),
|
||||
NOOP_QUERYWATCHER
|
||||
);
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue