Merge pull request #16501 from s1monw/trash_thread_local

Detach QueryShardContext from IndexShard and remove obsolete threadlocals
This commit is contained in:
Simon Willnauer 2016-02-08 21:01:35 +01:00
commit 3dd827017c
54 changed files with 226 additions and 285 deletions

View File

@ -164,7 +164,8 @@ public class TransportValidateQueryAction extends TransportBroadcastAction<Valid
protected ShardValidateQueryResponse shardOperation(ShardValidateQueryRequest request) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.getShard(request.shardId().id());
final QueryShardContext queryShardContext = indexShard.getQueryShardContext();
final QueryShardContext queryShardContext = indexService.newQueryShardContext();
queryShardContext.setTypes(request.types());
boolean valid;
String explanation = null;

View File

@ -121,7 +121,7 @@ public class TransportExplainAction extends TransportSingleShardAction<ExplainRe
SearchContext.setCurrent(context);
try {
context.parsedQuery(indexShard.getQueryShardContext().toQuery(request.query()));
context.parsedQuery(context.getQueryShardContext().toQuery(request.query()));
context.preProcess();
int topLevelDocId = result.docIdAndVersion().docId + result.docIdAndVersion().context.docBase;
Explanation explanation = context.searcher().explain(context.query(), topLevelDocId);

View File

@ -53,7 +53,6 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper;
@ -336,7 +335,7 @@ public class MetaDataCreateIndexService extends AbstractComponent {
}
}
QueryShardContext queryShardContext = indexService.getQueryShardContext();
final QueryShardContext queryShardContext = indexService.newQueryShardContext();
for (Alias alias : request.aliases()) {
if (Strings.hasLength(alias.filter())) {
aliasValidator.validateAliasFilter(alias.name(), alias.filter(), queryShardContext);

View File

@ -117,7 +117,7 @@ public class MetaDataIndexAliasesService extends AbstractComponent {
indices.put(indexMetaData.getIndex().getName(), indexService);
}
aliasValidator.validateAliasFilter(aliasAction.alias(), filter, indexService.getQueryShardContext());
aliasValidator.validateAliasFilter(aliasAction.alias(), filter, indexService.newQueryShardContext());
}
AliasMetaData newAliasMd = AliasMetaData.newAliasMetaDataBuilder(
aliasAction.alias())

View File

@ -131,7 +131,7 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
this.indexSettings = indexSettings;
this.analysisService = registry.build(indexSettings);
this.similarityService = similarityService;
this.mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, IndexService.this::getQueryShardContext);
this.mapperService = new MapperService(indexSettings, analysisService, similarityService, mapperRegistry, IndexService.this::newQueryShardContext);
this.indexFieldData = new IndexFieldDataService(indexSettings, indicesFieldDataCache, nodeServicesProvider.getCircuitBreakerService(), mapperService);
this.shardStoreDeleter = shardStoreDeleter;
this.eventListener = eventListener;
@ -417,7 +417,10 @@ public final class IndexService extends AbstractIndexComponent implements IndexC
return indexSettings;
}
public QueryShardContext getQueryShardContext() {
/**
* Creates a new QueryShardContext. The context has not types set yet, if types are required set them via {@link QueryShardContext#setTypes(String...)}
*/
public QueryShardContext newQueryShardContext() {
return new QueryShardContext(indexSettings, nodeServicesProvider.getClient(), indexCache.bitsetFilterCache(), indexFieldData, mapperService(), similarityService(), nodeServicesProvider.getScriptService(), nodeServicesProvider.getIndicesQueriesRegistry());
}

View File

@ -136,11 +136,11 @@ public final class SearchSlowLog {
public String toString() {
StringBuilder sb = new StringBuilder();
sb.append("took[").append(TimeValue.timeValueNanos(tookInNanos)).append("], took_millis[").append(TimeUnit.NANOSECONDS.toMillis(tookInNanos)).append("], ");
if (context.types() == null) {
if (context.getQueryShardContext().getTypes() == null) {
sb.append("types[], ");
} else {
sb.append("types[");
Strings.arrayToDelimitedString(context.types(), ",", sb);
Strings.arrayToDelimitedString(context.getQueryShardContext().getTypes(), ",", sb);
sb.append("], ");
}
if (context.groupStats() == null) {

View File

@ -49,7 +49,6 @@ public class DocumentMapperParser {
final MapperService mapperService;
final AnalysisService analysisService;
private static final ESLogger logger = Loggers.getLogger(DocumentMapperParser.class);
private final SimilarityService similarityService;
private final Supplier<QueryShardContext> queryShardContextSupplier;

View File

@ -26,6 +26,7 @@ import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.join.JoinUtil;
import org.apache.lucene.search.join.ScoreMode;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.Queries;
@ -205,13 +206,15 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
@Override
protected Query doToQuery(QueryShardContext context) throws IOException {
String[] previousTypes = QueryShardContext.setTypesWithPrevious(type);
Query innerQuery;
final String[] previousTypes = context.getTypes();
context.setTypes(type);
try {
innerQuery = query.toQuery(context);
} finally {
QueryShardContext.setTypes(previousTypes);
context.setTypes(previousTypes);
}
if (innerQuery == null) {
return null;
}

View File

@ -22,6 +22,7 @@ import org.apache.lucene.search.BooleanClause;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.join.ScoreMode;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lucene.search.Queries;
@ -119,11 +120,12 @@ public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBu
@Override
protected Query doToQuery(QueryShardContext context) throws IOException {
Query innerQuery;
String[] previousTypes = QueryShardContext.setTypesWithPrevious(type);
String[] previousTypes = context.getTypes();
context.setTypes(type);
try {
innerQuery = query.toQuery(context);
} finally {
QueryShardContext.setTypes(previousTypes);
context.setTypes(previousTypes);
}
if (innerQuery == null) {

View File

@ -30,6 +30,7 @@ import org.elasticsearch.client.Client;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.xcontent.XContentFactory;
@ -74,7 +75,6 @@ import static java.util.Collections.unmodifiableMap;
*/
public class QueryShardContext {
private static final ThreadLocal<String[]> typesContext = new ThreadLocal<>();
private final MapperService mapperService;
private final ScriptService scriptService;
private final SimilarityService similarityService;
@ -82,23 +82,14 @@ public class QueryShardContext {
private final IndexFieldDataService indexFieldDataService;
private final IndexSettings indexSettings;
private final Client client;
private String[] types = Strings.EMPTY_ARRAY;
public static void setTypes(String[] types) {
typesContext.set(types);
public void setTypes(String... types) {
this.types = types;
}
public static String[] getTypes() {
return typesContext.get();
}
public static String[] setTypesWithPrevious(String... types) {
String[] old = typesContext.get();
setTypes(types);
return old;
}
public static void removeTypes() {
typesContext.remove();
public String[] getTypes() {
return types;
}
private final Map<String, Query> namedQueries = new HashMap<>();
@ -126,6 +117,7 @@ public class QueryShardContext {
public QueryShardContext(QueryShardContext source) {
this(source.indexSettings, source.client, source.bitsetFilterCache, source.indexFieldDataService, source.mapperService, source.similarityService, source.scriptService, source.indicesQueriesRegistry);
this.types = source.getTypes();
}

View File

@ -61,7 +61,7 @@ public class NestedInnerQueryParseSupport {
private ObjectMapper parentObjectMapper;
public NestedInnerQueryParseSupport(XContentParser parser, SearchContext searchContext) {
shardContext = searchContext.indexShard().getQueryShardContext();
shardContext = searchContext.getQueryShardContext();
parseContext = shardContext.parseContext();
shardContext.reset(parser);

View File

@ -27,7 +27,6 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.UsageTrackingQueryCachingPolicy;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.ThreadInterruptedException;
import org.elasticsearch.ElasticsearchException;
@ -158,7 +157,6 @@ public class IndexShard extends AbstractIndexShardComponent {
private final IndicesQueryCache indicesQueryCache;
private final IndexEventListener indexEventListener;
private final IndexSettings idxSettings;
private final NodeServicesProvider provider;
/** How many bytes we are currently moving to disk, via either IndexWriter.flush or refresh. IndexingMemoryController polls this
* across all shards to decide if throttling is necessary because moving bytes to disk is falling behind vs incoming documents
@ -253,9 +251,9 @@ public class IndexShard extends AbstractIndexShardComponent {
this.engineConfig = newEngineConfig(translogConfig, cachingPolicy);
this.suspendableRefContainer = new SuspendableRefContainer();
this.provider = provider;
this.searcherWrapper = indexSearcherWrapper;
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, newQueryShardContext());
QueryShardContext queryShardContext = new QueryShardContext(idxSettings, provider.getClient(), indexCache.bitsetFilterCache(), indexFieldDataService, mapperService, similarityService, provider.getScriptService(), provider.getIndicesQueriesRegistry());
this.percolatorQueriesRegistry = new PercolatorQueriesRegistry(shardId, indexSettings, queryShardContext);
}
public Store store() {
@ -814,7 +812,7 @@ public class IndexShard extends AbstractIndexShardComponent {
engine.flushAndClose();
}
} finally { // playing safe here and close the engine even if the above succeeds - close can be called multiple times
IOUtils.close(engine, percolatorQueriesRegistry, queryShardContextCache);
IOUtils.close(engine, percolatorQueriesRegistry);
}
}
}
@ -1514,25 +1512,6 @@ public class IndexShard extends AbstractIndexShardComponent {
}
}
private CloseableThreadLocal<QueryShardContext> queryShardContextCache = new CloseableThreadLocal<QueryShardContext>() {
// TODO We should get rid of this threadlocal but I think it should be a sep change
@Override
protected QueryShardContext initialValue() {
return newQueryShardContext();
}
};
private QueryShardContext newQueryShardContext() {
return new QueryShardContext(idxSettings, provider.getClient(), indexCache.bitsetFilterCache(), indexFieldDataService, mapperService, similarityService, provider.getScriptService(), provider.getIndicesQueriesRegistry());
}
/**
* Returns a threadlocal QueryShardContext for this shard.
*/
public QueryShardContext getQueryShardContext() {
return queryShardContextCache.get();
}
EngineFactory getEngineFactory() {
return engineFactory;
}

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.percolator;
import com.carrotsearch.hppc.ObjectObjectAssociativeContainer;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.LeafReaderContext;
@ -32,7 +31,6 @@ import org.elasticsearch.action.percolate.PercolateShardRequest;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.util.BigArrays;
@ -46,6 +44,7 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
@ -73,11 +72,11 @@ import org.elasticsearch.search.query.QuerySearchResult;
import org.elasticsearch.search.rescore.RescoreSearchContext;
import org.elasticsearch.search.suggest.SuggestionSearchContext;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
/**
*/
@ -99,14 +98,11 @@ public class PercolateContext extends SearchContext {
private final long originNanoTime = System.nanoTime();
private final long startTime;
private final boolean onlyCount;
private String[] types;
private Engine.Searcher docSearcher;
private Engine.Searcher engineSearcher;
private ContextIndexSearcher searcher;
private SearchContextHighlight highlight;
private SearchLookup searchLookup;
private ParsedQuery parsedQuery;
private Query query;
private Query percolateQuery;
@ -115,7 +111,9 @@ public class PercolateContext extends SearchContext {
private QuerySearchResult querySearchResult;
private Sort sort;
private final Map<String, FetchSubPhaseContext> subPhaseContexts = new HashMap<>();
private final QueryShardContext queryShardContext;
private final Map<Class<?>, Collector> queryCollectors = new HashMap<>();
private SearchLookup searchLookup;
public PercolateContext(PercolateShardRequest request, SearchShardTarget searchShardTarget, IndexShard indexShard,
IndexService indexService, PageCacheRecycler pageCacheRecycler,
@ -126,7 +124,6 @@ public class PercolateContext extends SearchContext {
this.fieldDataService = indexService.fieldData();
this.mapperService = indexService.mapperService();
this.searchShardTarget = searchShardTarget;
this.types = new String[]{request.documentType()};
this.pageCacheRecycler = pageCacheRecycler;
this.bigArrays = bigArrays.withCircuitBreaking();
this.querySearchResult = new QuerySearchResult(0, searchShardTarget);
@ -137,10 +134,12 @@ public class PercolateContext extends SearchContext {
this.aliasFilter = aliasFilter;
this.startTime = request.getStartTime();
this.onlyCount = request.onlyCount();
queryShardContext = indexService.newQueryShardContext();
queryShardContext.setTypes(request.documentType());
}
// for testing:
PercolateContext(PercolateShardRequest request, SearchShardTarget searchShardTarget, MapperService mapperService) {
PercolateContext(PercolateShardRequest request, SearchShardTarget searchShardTarget, MapperService mapperService, QueryShardContext queryShardContext) {
super(null);
this.searchShardTarget = searchShardTarget;
this.mapperService = mapperService;
@ -154,6 +153,7 @@ public class PercolateContext extends SearchContext {
this.startTime = 0;
this.numberOfShards = 0;
this.onlyCount = true;
this.queryShardContext = queryShardContext;
}
public IndexSearcher docSearcher() {
@ -162,10 +162,10 @@ public class PercolateContext extends SearchContext {
public void initialize(Engine.Searcher docSearcher, ParsedDocument parsedDocument) {
this.docSearcher = docSearcher;
IndexReader indexReader = docSearcher.reader();
LeafReaderContext atomicReaderContext = indexReader.leaves().get(0);
LeafSearchLookup leafLookup = lookup().getLeafSearchLookup(atomicReaderContext);
this.searchLookup = new SearchLookup(mapperService(), fieldData(), queryShardContext.getTypes());
LeafSearchLookup leafLookup = searchLookup.getLeafSearchLookup(atomicReaderContext);
leafLookup.setDocument(0);
leafLookup.source().setSource(parsedDocument.source());
@ -232,10 +232,10 @@ public class PercolateContext extends SearchContext {
@Override
public SearchLookup lookup() {
if (searchLookup == null) {
searchLookup = new SearchLookup(mapperService(), fieldData(), types);
}
return searchLookup;
// we cache this since it's really just a single document lookup - check the init method for details
assert searchLookup != null : "context is not initialized";
assert Arrays.equals(searchLookup.doc().getTypes(), getQueryShardContext().getTypes()) : "types mismatch - can't return lookup";
return this.searchLookup;
}
@Override
@ -265,16 +265,6 @@ public class PercolateContext extends SearchContext {
return query;
}
@Override
public String[] types() {
return types;
}
public void types(String[] types) {
this.types = types;
searchLookup = new SearchLookup(mapperService(), fieldData(), types);
}
@Override
public IndexFieldDataService fieldData() {
return fieldDataService;
@ -341,11 +331,6 @@ public class PercolateContext extends SearchContext {
return numberOfShards;
}
@Override
public boolean hasTypes() {
throw new UnsupportedOperationException();
}
@Override
public float queryBoost() {
throw new UnsupportedOperationException();
@ -684,6 +669,11 @@ public class PercolateContext extends SearchContext {
return queryCollectors;
}
@Override
public QueryShardContext getQueryShardContext() {
return queryShardContext;
}
@Override
public Profilers getProfilers() {
throw new UnsupportedOperationException();

View File

@ -23,8 +23,6 @@ import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Query;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.action.percolate.PercolateShardRequest;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.BytesStreamOutput;
@ -35,7 +33,6 @@ import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.mapper.DocumentMapperForType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.ParsedDocument;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.search.SearchParseElement;
import org.elasticsearch.search.aggregations.AggregationPhase;
@ -60,7 +57,7 @@ public class PercolateDocumentParser {
this.aggregationPhase = aggregationPhase;
}
public ParsedDocument parse(PercolateShardRequest request, PercolateContext context, MapperService mapperService, QueryShardContext queryShardContext) {
public ParsedDocument parse(final PercolateShardRequest request, final PercolateContext context, final MapperService mapperService) {
BytesReference source = request.source();
if (source == null || source.length() == 0) {
if (request.docSource() != null && request.docSource().length() != 0) {
@ -73,13 +70,13 @@ public class PercolateDocumentParser {
// TODO: combine all feature parse elements into one map
Map<String, ? extends SearchParseElement> hlElements = highlightPhase.parseElements();
Map<String, ? extends SearchParseElement> aggregationElements = aggregationPhase.parseElements();
final QueryShardContext queryShardContext = context.getQueryShardContext();
ParsedDocument doc = null;
// Some queries (function_score query when for decay functions) rely on a SearchContext being set:
// We switch types because this context needs to be in the context of the percolate queries in the shard and
// not the in memory percolate doc
String[] previousTypes = context.types();
context.types(new String[]{PercolatorService.TYPE_NAME});
final String[] previousTypes = queryShardContext.getTypes();
queryShardContext.setTypes(PercolatorService.TYPE_NAME);
try (XContentParser parser = XContentFactory.xContent(source).createParser(source);) {
String currentFieldName = null;
XContentParser.Token token;
@ -176,7 +173,7 @@ public class PercolateDocumentParser {
} catch (Throwable e) {
throw new ElasticsearchParseException("failed to parse request", e);
} finally {
context.types(previousTypes);
queryShardContext.setTypes(previousTypes);
}
if (request.docSource() != null && request.docSource().length() != 0) {

View File

@ -169,8 +169,8 @@ public class PercolatorService extends AbstractComponent implements Releasable {
}
public PercolateShardResponse percolate(PercolateShardRequest request) throws IOException {
IndexService percolateIndexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = percolateIndexService.getShard(request.shardId().id());
final IndexService percolateIndexService = indicesService.indexServiceSafe(request.shardId().getIndex());
final IndexShard indexShard = percolateIndexService.getShard(request.shardId().id());
indexShard.readAllowed(); // check if we can read the shard...
PercolatorQueriesRegistry percolateQueryRegistry = indexShard.percolateRegistry();
percolateQueryRegistry.prePercolate();
@ -183,7 +183,7 @@ public class PercolatorService extends AbstractComponent implements Releasable {
indexShard.shardId().getIndex().getName(),
request.indices()
);
Query aliasFilter = percolateIndexService.aliasFilter(indexShard.getQueryShardContext(), filteringAliases);
Query aliasFilter = percolateIndexService.aliasFilter(percolateIndexService.newQueryShardContext(), filteringAliases);
SearchShardTarget searchShardTarget = new SearchShardTarget(clusterService.localNode().id(), request.shardId().getIndex(), request.shardId().id());
final PercolateContext context = new PercolateContext(
@ -191,8 +191,7 @@ public class PercolatorService extends AbstractComponent implements Releasable {
);
SearchContext.setCurrent(context);
try {
ParsedDocument parsedDocument = percolateDocumentParser.parse(request, context, percolateIndexService.mapperService(), percolateIndexService.getQueryShardContext());
ParsedDocument parsedDocument = percolateDocumentParser.parse(request, context, percolateIndexService.mapperService());
if (context.searcher().getIndexReader().maxDoc() == 0) {
return new PercolateShardResponse(Lucene.EMPTY_TOP_DOCS, Collections.emptyMap(), Collections.emptyMap(), context);
}

View File

@ -646,8 +646,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
if (source == null) {
return;
}
final IndexShard indexShard = context.indexShard();
QueryShardContext queryShardContext = indexShard.getQueryShardContext();
QueryShardContext queryShardContext = context.getQueryShardContext();
context.from(source.from());
context.size(source.size());
ObjectFloatHashMap<String> indexBoostMap = source.indexBoost();
@ -751,7 +750,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
if (source.rescores() != null) {
try {
for (RescoreBuilder<?> rescore : source.rescores()) {
context.addRescore(rescore.build(context.indexShard().getQueryShardContext()));
context.addRescore(rescore.build(context.getQueryShardContext()));
}
} catch (IOException e) {
throw new SearchContextException(context, "failed to create RescoreSearchContext", e);
@ -776,7 +775,7 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
if (source.highlighter() != null) {
HighlightBuilder highlightBuilder = source.highlighter();
try {
context.highlight(highlightBuilder.build(context.indexShard().getQueryShardContext()));
context.highlight(highlightBuilder.build(context.getQueryShardContext()));
} catch (IOException e) {
throw new SearchContextException(context, "failed to create SearchContextHighlighter", e);
}

View File

@ -124,7 +124,7 @@ public class AggregationPhase implements SearchPhase {
if (!globals.isEmpty()) {
BucketCollector globalsCollector = BucketCollector.wrap(globals);
Query query = Queries.newMatchAllQuery();
Query searchFilter = context.searchFilter(context.types());
Query searchFilter = context.searchFilter(context.getQueryShardContext().getTypes());
if (searchFilter != null) {
BooleanQuery filtered = new BooleanQuery.Builder()

View File

@ -39,7 +39,7 @@ public class FilterParser implements Aggregator.Parser {
@Override
public AggregatorFactory parse(String aggregationName, XContentParser parser, SearchContext context) throws IOException {
ParsedQuery filter = context.indexShard().getQueryShardContext().parseInnerFilter(parser);
ParsedQuery filter = context.getQueryShardContext().parseInnerFilter(parser);
return new FilterAggregator.Factory(aggregationName, filter == null ? new MatchAllDocsQuery() : filter.query());
}

View File

@ -82,7 +82,7 @@ public class FiltersParser implements Aggregator.Parser {
if (token == XContentParser.Token.FIELD_NAME) {
key = parser.currentName();
} else {
ParsedQuery filter = context.indexShard().getQueryShardContext().parseInnerFilter(parser);
ParsedQuery filter = context.getQueryShardContext().parseInnerFilter(parser);
filters.add(new FiltersAggregator.KeyedFilter(key, filter == null ? Queries.newMatchAllQuery() : filter.query()));
}
}
@ -95,7 +95,7 @@ public class FiltersParser implements Aggregator.Parser {
keyed = false;
int idx = 0;
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
ParsedQuery filter = context.indexShard().getQueryShardContext().parseInnerFilter(parser);
ParsedQuery filter = context.getQueryShardContext().parseInnerFilter(parser);
filters.add(new FiltersAggregator.KeyedFilter(String.valueOf(idx), filter == null ? Queries.newMatchAllQuery()
: filter.query()));
idx++;

View File

@ -66,7 +66,7 @@ public class SignificantTermsParametersParser extends AbstractTermsParametersPar
if (significanceHeuristicParser != null) {
significanceHeuristic = significanceHeuristicParser.parse(parser, context.parseFieldMatcher(), context);
} else if (context.parseFieldMatcher().match(currentFieldName, BACKGROUND_FILTER)) {
filter = context.indexShard().getQueryShardContext().parseInnerFilter(parser).query();
filter = context.getQueryShardContext().parseInnerFilter(parser).query();
} else {
throw new SearchParseException(context, "Unknown key for a " + token + " in [" + aggregationName + "]: ["
+ currentFieldName + "].", parser.getTokenLocation());

View File

@ -59,7 +59,7 @@ public class InnerHitsParseElement implements SearchParseElement {
@Override
public void parse(XContentParser parser, SearchContext searchContext) throws Exception {
QueryShardContext context = searchContext.indexShard().getQueryShardContext();
QueryShardContext context = searchContext.getQueryShardContext();
context.reset(parser);
Map<String, InnerHitsContext.BaseInnerHits> topLevelInnerHits = parseInnerHits(parser, context, searchContext);
if (topLevelInnerHits != null) {

View File

@ -54,7 +54,7 @@ public class HighlighterParseElement implements SearchParseElement {
@Override
public void parse(XContentParser parser, SearchContext context) throws Exception {
try {
context.highlight(parse(parser, context.indexShard().getQueryShardContext()));
context.highlight(parse(parser, context.getQueryShardContext()));
} catch (IllegalArgumentException ex) {
throw new SearchParseException(context, "Error while trying to parse Highlighter element in request", parser.getTokenLocation());
}

View File

@ -29,7 +29,6 @@ import org.apache.lucene.search.Query;
import org.apache.lucene.search.Sort;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.util.Counter;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
@ -53,6 +52,7 @@ import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.query.AbstractQueryBuilder;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
@ -62,7 +62,6 @@ import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.SearchContextHighlight;
@ -150,6 +149,7 @@ public class DefaultSearchContext extends SearchContext {
private final Map<String, FetchSubPhaseContext> subPhaseContexts = new HashMap<>();
private final Map<Class<?>, Collector> queryCollectors = new HashMap<>();
private final QueryShardContext queryShardContext;
public DefaultSearchContext(long id, ShardSearchRequest request, SearchShardTarget shardTarget,
Engine.Searcher engineSearcher, IndexService indexService, IndexShard indexShard,
@ -175,6 +175,8 @@ public class DefaultSearchContext extends SearchContext {
this.searcher = new ContextIndexSearcher(engineSearcher, indexService.cache().query(), indexShard.getQueryCachingPolicy());
this.timeEstimateCounter = timeEstimateCounter;
this.timeoutInMillis = timeout.millis();
queryShardContext = indexService.newQueryShardContext();
queryShardContext.setTypes(request.types());
}
@Override
@ -206,7 +208,7 @@ public class DefaultSearchContext extends SearchContext {
}
// initialize the filtering alias based on the provided filters
aliasFilter = indexService.aliasFilter(indexShard.getQueryShardContext(), request.filteringAliases());
aliasFilter = indexService.aliasFilter(queryShardContext, request.filteringAliases());
if (query() == null) {
parsedQuery(ParsedQuery.parsedMatchAllQuery());
@ -223,7 +225,7 @@ public class DefaultSearchContext extends SearchContext {
}
private ParsedQuery buildFilteredQuery() {
Query searchFilter = searchFilter(types());
Query searchFilter = searchFilter(queryShardContext.getTypes());
if (searchFilter == null) {
return originalQuery;
}
@ -312,16 +314,6 @@ public class DefaultSearchContext extends SearchContext {
return request.numberOfShards();
}
@Override
public boolean hasTypes() {
return request.types() != null && request.types().length > 0;
}
@Override
public String[] types() {
return request.types();
}
@Override
public float queryBoost() {
return queryBoost;
@ -765,6 +757,11 @@ public class DefaultSearchContext extends SearchContext {
return queryCollectors;
}
@Override
public QueryShardContext getQueryShardContext() {
return queryShardContext;
}
@Override
public Profilers getProfilers() {
return profilers;

View File

@ -35,6 +35,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
@ -117,16 +118,6 @@ public abstract class FilteredSearchContext extends SearchContext {
return in.numberOfShards();
}
@Override
public boolean hasTypes() {
return in.hasTypes();
}
@Override
public String[] types() {
return in.types();
}
@Override
public float queryBoost() {
return in.queryBoost();
@ -525,4 +516,8 @@ public abstract class FilteredSearchContext extends SearchContext {
@Override
public Map<Class<?>, Collector> queryCollectors() { return in.queryCollectors();}
@Override
public QueryShardContext getQueryShardContext() {
return in.getQueryShardContext();
}
}

View File

@ -72,12 +72,10 @@ public abstract class SearchContext implements Releasable {
public static void setCurrent(SearchContext value) {
current.set(value);
QueryShardContext.setTypes(value.types());
}
public static void removeCurrent() {
current.remove();
QueryShardContext.removeTypes();
}
public static SearchContext current() {
@ -134,10 +132,6 @@ public abstract class SearchContext implements Releasable {
public abstract int numberOfShards();
public abstract boolean hasTypes();
public abstract String[] types();
public abstract float queryBoost();
public abstract SearchContext queryBoost(float queryBoost);
@ -379,4 +373,6 @@ public abstract class SearchContext implements Releasable {
CONTEXT
}
public abstract QueryShardContext getQueryShardContext();
}

View File

@ -51,4 +51,8 @@ public class DocLookup {
public LeafDocLookup getLeafDocLookup(LeafReaderContext context) {
return new LeafDocLookup(mapperService, fieldDataService, types, context);
}
public String[] getTypes() {
return types;
}
}

View File

@ -30,9 +30,9 @@ public class PostFilterParseElement implements SearchParseElement {
@Override
public void parse(XContentParser parser, SearchContext context) throws Exception {
ParsedQuery postFilter = context.indexShard().getQueryShardContext().parseInnerFilter(parser);
ParsedQuery postFilter = context.getQueryShardContext().parseInnerFilter(parser);
if (postFilter != null) {
context.parsedPostFilter(postFilter);
}
}
}
}

View File

@ -30,6 +30,6 @@ public class QueryParseElement implements SearchParseElement {
@Override
public void parse(XContentParser parser, SearchContext context) throws Exception {
context.parsedQuery(context.indexShard().getQueryShardContext().parse(parser));
context.parsedQuery(context.getQueryShardContext().parse(parser));
}
}
}

View File

@ -36,10 +36,10 @@ public class RescoreParseElement implements SearchParseElement {
public void parse(XContentParser parser, SearchContext context) throws Exception {
if (parser.currentToken() == XContentParser.Token.START_ARRAY) {
while (parser.nextToken() != XContentParser.Token.END_ARRAY) {
context.addRescore(parseSingleRescoreContext(parser, context.indexShard().getQueryShardContext()));
context.addRescore(parseSingleRescoreContext(parser, context.getQueryShardContext()));
}
} else {
context.addRescore(parseSingleRescoreContext(parser, context.indexShard().getQueryShardContext()));
context.addRescore(parseSingleRescoreContext(parser, context.getQueryShardContext()));
}
}

View File

@ -33,7 +33,6 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.script.CompiledScript;
import org.elasticsearch.script.ExecutableScript;
@ -65,7 +64,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
/*
* More Ideas:
* - add ability to find whitespace problems -> we can build a poor mans decompounder with our index based on a automaton?
* - add ability to build different error models maybe based on a confusion matrix?
* - add ability to build different error models maybe based on a confusion matrix?
* - try to combine a token with its subsequent token to find / detect word splits (optional)
* - for this to work we need some way to defined the position length of a candidate
* - phonetic filters could be interesting here too for candidate selection
@ -84,8 +83,8 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
DirectSpellChecker directSpellChecker = SuggestUtils.getDirectSpellChecker(generator);
Terms terms = MultiFields.getTerms(indexReader, generator.field());
if (terms != null) {
gens.add(new DirectCandidateGenerator(directSpellChecker, generator.field(), generator.suggestMode(),
indexReader, realWordErrorLikelihood, generator.size(), generator.preFilter(), generator.postFilter(), terms));
gens.add(new DirectCandidateGenerator(directSpellChecker, generator.field(), generator.suggestMode(),
indexReader, realWordErrorLikelihood, generator.size(), generator.preFilter(), generator.postFilter(), terms));
}
}
final String suggestField = suggestion.getField();
@ -119,8 +118,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
final ExecutableScript executable = scriptService.executable(collateScript, vars);
final BytesReference querySource = (BytesReference) executable.run();
IndexService indexService = indicesService.indexService(suggestion.getIndex());
IndexShard shard = indexService.getShard(suggestion.getShard());
final ParsedQuery parsedQuery = shard.getQueryShardContext().parse(querySource);
final ParsedQuery parsedQuery = indexService.newQueryShardContext().parse(querySource);
collateMatch = Lucene.exists(searcher, parsedQuery.query());
}
if (!collateMatch && !collatePrune) {
@ -152,7 +150,7 @@ public final class PhraseSuggester extends Suggester<PhraseSuggestionContext> {
ScriptService scriptService() {
return scriptService;
}
@Override
public SuggestContextParser getContextParser() {
return new PhraseSuggestParser(this);

View File

@ -42,7 +42,6 @@ class PhraseSuggestionContext extends SuggestionContext {
private BytesRef preTag;
private BytesRef postTag;
private CompiledScript collateQueryScript;
private CompiledScript collateFilterScript;
private Map<String, Object> collateScriptParams = new HashMap<>(1);
private WordScorer.WordScorerFactory scorer;

View File

@ -194,7 +194,7 @@ public class ESExceptionTests extends ESTestCase {
public void testToXContent() throws IOException {
{
ElasticsearchException ex = new SearchParseException(new TestSearchContext(), "foo", new XContentLocation(1,0));
ElasticsearchException ex = new SearchParseException(new TestSearchContext(null), "foo", new XContentLocation(1,0));
XContentBuilder builder = XContentFactory.jsonBuilder();
builder.startObject();
ex.toXContent(builder, PARAMS);

View File

@ -439,7 +439,7 @@ public class ExceptionSerializationTests extends ESTestCase {
}
public void testSearchParseException() throws IOException {
SearchContext ctx = new TestSearchContext();
SearchContext ctx = new TestSearchContext(null);
SearchParseException ex = serialize(new SearchParseException(ctx, "foo", new XContentLocation(66, 666)));
assertEquals("foo", ex.getMessage());
assertEquals(66, ex.getLineNumber());

View File

@ -39,7 +39,6 @@ import org.elasticsearch.test.ESSingleNodeTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
@ -87,17 +86,17 @@ public class IndexServiceTests extends ESSingleNodeTestCase {
assertThat(indexService.getMetaData().getAliases().containsKey("dogs"), equalTo(true));
assertThat(indexService.getMetaData().getAliases().containsKey("turtles"), equalTo(false));
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "cats").toString(), equalTo("animal:cat"));
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "cats", "dogs").toString(), equalTo("animal:cat animal:dog"));
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "cats").toString(), equalTo("animal:cat"));
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "cats", "dogs").toString(), equalTo("animal:cat animal:dog"));
// Non-filtering alias should turn off all filters because filters are ORed
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "all"), nullValue());
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "cats", "all"), nullValue());
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "all", "cats"), nullValue());
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "all"), nullValue());
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "cats", "all"), nullValue());
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "all", "cats"), nullValue());
add(indexService, "cats", filter(termQuery("animal", "feline")));
add(indexService, "dogs", filter(termQuery("animal", "canine")));
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "dogs", "cats").toString(), equalTo("animal:canine animal:feline"));
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "dogs", "cats").toString(), equalTo("animal:canine animal:feline"));
}
public void testAliasFilters() throws Exception {
@ -107,14 +106,14 @@ public class IndexServiceTests extends ESSingleNodeTestCase {
add(indexService, "cats", filter(termQuery("animal", "cat")));
add(indexService, "dogs", filter(termQuery("animal", "dog")));
assertThat(indexService.aliasFilter(shard.getQueryShardContext()), nullValue());
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "dogs").toString(), equalTo("animal:dog"));
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "dogs", "cats").toString(), equalTo("animal:dog animal:cat"));
assertThat(indexService.aliasFilter(indexService.newQueryShardContext()), nullValue());
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "dogs").toString(), equalTo("animal:dog"));
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "dogs", "cats").toString(), equalTo("animal:dog animal:cat"));
add(indexService, "cats", filter(termQuery("animal", "feline")));
add(indexService, "dogs", filter(termQuery("animal", "canine")));
assertThat(indexService.aliasFilter(shard.getQueryShardContext(), "dogs", "cats").toString(), equalTo("animal:canine animal:feline"));
assertThat(indexService.aliasFilter(indexService.newQueryShardContext(), "dogs", "cats").toString(), equalTo("animal:canine animal:feline"));
}
public void testRemovedAliasFilter() throws Exception {
@ -124,7 +123,7 @@ public class IndexServiceTests extends ESSingleNodeTestCase {
add(indexService, "cats", filter(termQuery("animal", "cat")));
remove(indexService, "cats");
try {
indexService.aliasFilter(shard.getQueryShardContext(), "cats");
indexService.aliasFilter(indexService.newQueryShardContext(), "cats");
fail("Expected InvalidAliasNameException");
} catch (InvalidAliasNameException e) {
assertThat(e.getMessage(), containsString("Invalid alias name [cats]"));
@ -139,7 +138,7 @@ public class IndexServiceTests extends ESSingleNodeTestCase {
add(indexService, "dogs", filter(termQuery("animal", "dog")));
try {
indexService.aliasFilter(shard.getQueryShardContext(), "unknown");
indexService.aliasFilter(indexService.newQueryShardContext(), "unknown");
fail();
} catch (InvalidAliasNameException e) {
// all is well

View File

@ -260,7 +260,7 @@ public class SimpleDateMappingTests extends ESSingleNodeTestCase {
NumericRangeQuery<Long> rangeQuery;
try {
SearchContext.setCurrent(new TestSearchContext());
SearchContext.setCurrent(new TestSearchContext(null));
rangeQuery = (NumericRangeQuery<Long>) defaultMapper.mappers().smartNameFieldMapper("date_field").fieldType().rangeQuery("10:00:00", "11:00:00", true, true).rewrite(null);
} finally {
SearchContext.removeCurrent();
@ -286,7 +286,7 @@ public class SimpleDateMappingTests extends ESSingleNodeTestCase {
NumericRangeQuery<Long> rangeQuery;
try {
SearchContext.setCurrent(new TestSearchContext());
SearchContext.setCurrent(new TestSearchContext(null));
rangeQuery = (NumericRangeQuery<Long>) defaultMapper.mappers().smartNameFieldMapper("date_field").fieldType().rangeQuery("Jan 02 10:00:00", "Jan 02 11:00:00", true, true).rewrite(null);
} finally {
SearchContext.removeCurrent();

View File

@ -63,7 +63,7 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
Collections.singletonMap(ExternalMetadataMapper.CONTENT_TYPE, new ExternalMetadataMapper.TypeParser()));
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
DocumentMapper documentMapper = parser.parse("type", new CompressedXContent(
XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject(ExternalMetadataMapper.CONTENT_TYPE)
@ -109,7 +109,7 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap());
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
DocumentMapper documentMapper = parser.parse("type", new CompressedXContent(
XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties")
@ -168,7 +168,7 @@ public class SimpleExternalMappingTests extends ESSingleNodeTestCase {
MapperRegistry mapperRegistry = new MapperRegistry(mapperParsers, Collections.emptyMap());
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
DocumentMapper documentMapper = parser.parse("type", new CompressedXContent(
XContentFactory.jsonBuilder().startObject().startObject("type").startObject("properties")

View File

@ -236,9 +236,9 @@ public class FieldNamesFieldMapperTests extends ESSingleNodeTestCase {
IndicesModule indicesModule = new IndicesModule();
indicesModule.registerMetadataMapper("_dummy", new DummyMetadataFieldMapper.TypeParser());
final MapperRegistry mapperRegistry = indicesModule.getMapperRegistry();
MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
MapperService mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
DocumentMapperParser parser = new DocumentMapperParser(indexService.getIndexSettings(), mapperService,
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type").endObject().endObject().string();
DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping));
ParsedDocument parsedDocument = mapper.parse("index", "type", "id", new BytesArray("{}"));

View File

@ -332,15 +332,14 @@ public abstract class AbstractQueryTestCase<QB extends AbstractQueryBuilder<QB>>
}
protected void setSearchContext(String[] types) {
TestSearchContext testSearchContext = new TestSearchContext();
testSearchContext.setTypes(types);
TestSearchContext testSearchContext = new TestSearchContext(queryShardContext);
testSearchContext.getQueryShardContext().setTypes(types);
SearchContext.setCurrent(testSearchContext);
}
@After
public void afterTest() {
clientInvocationHandler.delegate = null;
QueryShardContext.removeTypes();
SearchContext.removeCurrent();
}

View File

@ -84,7 +84,7 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
protected void setSearchContext(String[] types) {
final MapperService mapperService = queryShardContext().getMapperService();
final IndexFieldDataService fieldData = indexFieldDataService();
TestSearchContext testSearchContext = new TestSearchContext() {
TestSearchContext testSearchContext = new TestSearchContext(queryShardContext()) {
@Override
public MapperService mapperService() {
@ -96,7 +96,7 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
return fieldData; // need to build / parse inner hits sort fields
}
};
testSearchContext.setTypes(types);
testSearchContext.getQueryShardContext().setTypes(types);
SearchContext.setCurrent(testSearchContext);
}
@ -230,11 +230,12 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
}
public void testToQueryInnerQueryType() throws IOException {
String[] searchTypes = new String[]{PARENT_TYPE};
QueryShardContext.setTypes(searchTypes);
QueryShardContext shardContext = createShardContext();
shardContext.setTypes(searchTypes);
HasChildQueryBuilder hasChildQueryBuilder = new HasChildQueryBuilder(CHILD_TYPE, new IdsQueryBuilder().addIds("id"));
Query query = hasChildQueryBuilder.toQuery(createShardContext());
Query query = hasChildQueryBuilder.toQuery(shardContext);
//verify that the context types are still the same as the ones we previously set
assertThat(QueryShardContext.getTypes(), equalTo(searchTypes));
assertThat(shardContext.getTypes(), equalTo(searchTypes));
assertLateParsingQuery(query, CHILD_TYPE, "id");
}
@ -253,7 +254,7 @@ public class HasChildQueryBuilderTests extends AbstractQueryTestCase<HasChildQue
ConstantScoreQuery constantScoreQuery = (ConstantScoreQuery) rewrittenTermsQuery;
assertThat(constantScoreQuery.getQuery(), instanceOf(BooleanQuery.class));
BooleanQuery booleanTermsQuery = (BooleanQuery) constantScoreQuery.getQuery();
assertThat(booleanTermsQuery.clauses().size(), equalTo(1));
assertThat(booleanTermsQuery.clauses().toString(), booleanTermsQuery.clauses().size(), equalTo(1));
assertThat(booleanTermsQuery.clauses().get(0).getOccur(), equalTo(BooleanClause.Occur.SHOULD));
assertThat(booleanTermsQuery.clauses().get(0).getQuery(), instanceOf(TermQuery.class));
TermQuery termQuery = (TermQuery) booleanTermsQuery.clauses().get(0).getQuery();

View File

@ -79,7 +79,7 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
protected void setSearchContext(String[] types) {
final MapperService mapperService = queryShardContext().getMapperService();
final IndexFieldDataService fieldData = indexFieldDataService();
TestSearchContext testSearchContext = new TestSearchContext() {
TestSearchContext testSearchContext = new TestSearchContext(queryShardContext()) {
@Override
public MapperService mapperService() {
@ -91,7 +91,7 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
return fieldData; // need to build / parse inner hits sort fields
}
};
testSearchContext.setTypes(types);
testSearchContext.getQueryShardContext().setTypes(types);
SearchContext.setCurrent(testSearchContext);
}
@ -192,11 +192,12 @@ public class HasParentQueryBuilderTests extends AbstractQueryTestCase<HasParentQ
public void testToQueryInnerQueryType() throws IOException {
String[] searchTypes = new String[]{CHILD_TYPE};
QueryShardContext.setTypes(searchTypes);
QueryShardContext shardContext = createShardContext();
shardContext.setTypes(searchTypes);
HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(PARENT_TYPE, new IdsQueryBuilder().addIds("id"));
Query query = hasParentQueryBuilder.toQuery(createShardContext());
Query query = hasParentQueryBuilder.toQuery(shardContext);
//verify that the context types are still the same as the ones we previously set
assertThat(QueryShardContext.getTypes(), equalTo(searchTypes));
assertThat(shardContext.getTypes(), equalTo(searchTypes));
HasChildQueryBuilderTests.assertLateParsingQuery(query, PARENT_TYPE, "id");
}

View File

@ -60,7 +60,7 @@ public class NestedQueryBuilderTests extends AbstractQueryTestCase<NestedQueryBu
protected void setSearchContext(String[] types) {
final MapperService mapperService = queryShardContext().getMapperService();
final IndexFieldDataService fieldData = indexFieldDataService();
TestSearchContext testSearchContext = new TestSearchContext() {
TestSearchContext testSearchContext = new TestSearchContext(queryShardContext()) {
@Override
public MapperService mapperService() {
@ -72,7 +72,7 @@ public class NestedQueryBuilderTests extends AbstractQueryTestCase<NestedQueryBu
return fieldData; // need to build / parse inner hits sort fields
}
};
testSearchContext.setTypes(types);
testSearchContext.getQueryShardContext().setTypes(types);
SearchContext.setCurrent(testSearchContext);
}
@ -139,45 +139,45 @@ public class NestedQueryBuilderTests extends AbstractQueryTestCase<NestedQueryBu
public void testFromJson() throws IOException {
String json =
"{\n" +
" \"nested\" : {\n" +
" \"query\" : {\n" +
" \"bool\" : {\n" +
" \"must\" : [ {\n" +
" \"match\" : {\n" +
" \"obj1.name\" : {\n" +
" \"query\" : \"blue\",\n" +
" \"type\" : \"boolean\",\n" +
" \"operator\" : \"OR\",\n" +
" \"slop\" : 0,\n" +
" \"prefix_length\" : 0,\n" +
" \"max_expansions\" : 50,\n" +
" \"fuzzy_transpositions\" : true,\n" +
" \"lenient\" : false,\n" +
" \"zero_terms_query\" : \"NONE\",\n" +
" \"boost\" : 1.0\n" +
" }\n" +
" }\n" +
" }, {\n" +
" \"range\" : {\n" +
" \"obj1.count\" : {\n" +
" \"from\" : 5,\n" +
" \"to\" : null,\n" +
" \"include_lower\" : false,\n" +
" \"include_upper\" : true,\n" +
" \"boost\" : 1.0\n" +
" }\n" +
" }\n" +
" } ],\n" +
" \"disable_coord\" : false,\n" +
" \"adjust_pure_negative\" : true,\n" +
" \"boost\" : 1.0\n" +
" }\n" +
" },\n" +
" \"path\" : \"obj1\",\n" +
" \"score_mode\" : \"avg\",\n" +
" \"boost\" : 1.0\n" +
" }\n" +
"{\n" +
" \"nested\" : {\n" +
" \"query\" : {\n" +
" \"bool\" : {\n" +
" \"must\" : [ {\n" +
" \"match\" : {\n" +
" \"obj1.name\" : {\n" +
" \"query\" : \"blue\",\n" +
" \"type\" : \"boolean\",\n" +
" \"operator\" : \"OR\",\n" +
" \"slop\" : 0,\n" +
" \"prefix_length\" : 0,\n" +
" \"max_expansions\" : 50,\n" +
" \"fuzzy_transpositions\" : true,\n" +
" \"lenient\" : false,\n" +
" \"zero_terms_query\" : \"NONE\",\n" +
" \"boost\" : 1.0\n" +
" }\n" +
" }\n" +
" }, {\n" +
" \"range\" : {\n" +
" \"obj1.count\" : {\n" +
" \"from\" : 5,\n" +
" \"to\" : null,\n" +
" \"include_lower\" : false,\n" +
" \"include_upper\" : true,\n" +
" \"boost\" : 1.0\n" +
" }\n" +
" }\n" +
" } ],\n" +
" \"disable_coord\" : false,\n" +
" \"adjust_pure_negative\" : true,\n" +
" \"boost\" : 1.0\n" +
" }\n" +
" },\n" +
" \"path\" : \"obj1\",\n" +
" \"score_mode\" : \"avg\",\n" +
" \"boost\" : 1.0\n" +
" }\n" +
"}";
NestedQueryBuilder parsed = (NestedQueryBuilder) parseQuery(json);

View File

@ -64,7 +64,7 @@ public class ParentIdQueryBuilderTests extends AbstractQueryTestCase<ParentIdQue
protected void setSearchContext(String[] types) {
final MapperService mapperService = queryShardContext().getMapperService();
final IndexFieldDataService fieldData = indexFieldDataService();
TestSearchContext testSearchContext = new TestSearchContext() {
TestSearchContext testSearchContext = new TestSearchContext(queryShardContext()) {
@Override
public MapperService mapperService() {
@ -76,7 +76,7 @@ public class ParentIdQueryBuilderTests extends AbstractQueryTestCase<ParentIdQue
return fieldData; // need to build / parse inner hits sort fields
}
};
testSearchContext.setTypes(types);
testSearchContext.getQueryShardContext().setTypes(types);
SearchContext.setCurrent(testSearchContext);
}

View File

@ -68,7 +68,7 @@ public class CustomQueryParserIT extends ESIntegTestCase {
private static QueryShardContext queryShardContext() {
IndicesService indicesService = internalCluster().getDataNodeInstance(IndicesService.class);
return indicesService.indexServiceSafe("index").getQueryShardContext();
return indicesService.indexServiceSafe("index").newQueryShardContext();
}
//see #11120

View File

@ -69,7 +69,7 @@ public class MultiMatchQueryTests extends ESSingleNodeTestCase {
}
public void testCrossFieldMultiMatchQuery() throws IOException {
QueryShardContext queryShardContext = indexService.getShard(0).getQueryShardContext();
QueryShardContext queryShardContext = indexService.newQueryShardContext();
queryShardContext.setAllowUnmappedFields(true);
Query parsedQuery = multiMatchQuery("banon").field("name.first", 2).field("name.last", 3).field("foobar").type(MultiMatchQueryBuilder.Type.CROSS_FIELDS).toQuery(queryShardContext);
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {

View File

@ -104,8 +104,8 @@ public class PercolateDocumentParserTests extends ESTestCase {
.endObject();
Mockito.when(request.source()).thenReturn(source.bytes());
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService);
ParsedDocument parsedDocument = parser.parse(request, context, mapperService, queryShardContext);
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService, queryShardContext);
ParsedDocument parsedDocument = parser.parse(request, context, mapperService);
assertThat(parsedDocument.rootDoc().get("field1"), equalTo("value1"));
}
@ -123,8 +123,8 @@ public class PercolateDocumentParserTests extends ESTestCase {
.endObject();
Mockito.when(request.source()).thenReturn(source.bytes());
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService);
ParsedDocument parsedDocument = parser.parse(request, context, mapperService, queryShardContext);
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService, queryShardContext);
ParsedDocument parsedDocument = parser.parse(request, context, mapperService);
assertThat(parsedDocument.rootDoc().get("field1"), equalTo("value1"));
assertThat(context.percolateQuery(), equalTo(new TermQuery(new Term("field1", "value1"))));
assertThat(context.trackScores(), is(true));
@ -147,8 +147,8 @@ public class PercolateDocumentParserTests extends ESTestCase {
Mockito.when(request.source()).thenReturn(source.bytes());
Mockito.when(request.docSource()).thenReturn(docSource.bytes());
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService);
ParsedDocument parsedDocument = parser.parse(request, context, mapperService, queryShardContext);
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService, queryShardContext);
ParsedDocument parsedDocument = parser.parse(request, context, mapperService);
assertThat(parsedDocument.rootDoc().get("field1"), equalTo("value1"));
assertThat(context.percolateQuery(), equalTo(new TermQuery(new Term("field1", "value1"))));
assertThat(context.trackScores(), is(true));
@ -174,9 +174,9 @@ public class PercolateDocumentParserTests extends ESTestCase {
Mockito.when(request.source()).thenReturn(source.bytes());
Mockito.when(request.docSource()).thenReturn(docSource.bytes());
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService);
PercolateContext context = new PercolateContext(request, new SearchShardTarget("_node", new Index("_index", "_na_"), 0), mapperService, queryShardContext);
try {
parser.parse(request, context, mapperService, queryShardContext);
parser.parse(request, context, mapperService);
} catch (IllegalArgumentException e) {
assertThat(e.getMessage(), equalTo("Can't specify the document to percolate in the source of the request and as document id"));
}

View File

@ -1351,12 +1351,7 @@ public class PercolatorIT extends ESIntegTestCase {
assertThat(convertFromTextArray(response.getMatches(), "test"), arrayContainingInAnyOrder("1", "2", "3", "4", "5"));
PercolateResponse.Match[] matches = response.getMatches();
Arrays.sort(matches, new Comparator<PercolateResponse.Match>() {
@Override
public int compare(PercolateResponse.Match a, PercolateResponse.Match b) {
return a.getId().compareTo(b.getId());
}
});
Arrays.sort(matches, (a, b) -> a.getId().compareTo(b.getId()));
assertThat(matches[0].getHighlightFields().get("field1").fragments()[0].string(), equalTo("The quick <em>brown</em> <em>fox</em> jumps over the lazy dog"));
assertThat(matches[1].getHighlightFields().get("field1").fragments()[0].string(), equalTo("The quick brown fox jumps over the <em>lazy</em> <em>dog</em>"));

View File

@ -27,7 +27,7 @@ import org.elasticsearch.test.TestSearchContext;
public class GeoHashGridParserTests extends ESTestCase {
public void testParseValidFromInts() throws Exception {
SearchContext searchContext = new TestSearchContext();
SearchContext searchContext = new TestSearchContext(null);
int precision = randomIntBetween(1, 12);
XContentParser stParser = JsonXContent.jsonXContent.createParser(
"{\"field\":\"my_loc\", \"precision\":" + precision + ", \"size\": 500, \"shard_size\": 550}");
@ -37,7 +37,7 @@ public class GeoHashGridParserTests extends ESTestCase {
}
public void testParseValidFromStrings() throws Exception {
SearchContext searchContext = new TestSearchContext();
SearchContext searchContext = new TestSearchContext(null);
int precision = randomIntBetween(1, 12);
XContentParser stParser = JsonXContent.jsonXContent.createParser(
"{\"field\":\"my_loc\", \"precision\":\"" + precision + "\", \"size\": \"500\", \"shard_size\": \"550\"}");
@ -47,7 +47,7 @@ public class GeoHashGridParserTests extends ESTestCase {
}
public void testParseErrorOnNonIntPrecision() throws Exception {
SearchContext searchContext = new TestSearchContext();
SearchContext searchContext = new TestSearchContext(null);
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"my_loc\", \"precision\":\"2.0\"}");
GeoHashGridParser parser = new GeoHashGridParser();
try {
@ -59,7 +59,7 @@ public class GeoHashGridParserTests extends ESTestCase {
}
public void testParseErrorOnBooleanPrecision() throws Exception {
SearchContext searchContext = new TestSearchContext();
SearchContext searchContext = new TestSearchContext(null);
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"my_loc\", \"precision\":false}");
GeoHashGridParser parser = new GeoHashGridParser();
try {
@ -71,7 +71,7 @@ public class GeoHashGridParserTests extends ESTestCase {
}
public void testParseErrorOnPrecisionOutOfRange() throws Exception {
SearchContext searchContext = new TestSearchContext();
SearchContext searchContext = new TestSearchContext(null);
XContentParser stParser = JsonXContent.jsonXContent.createParser("{\"field\":\"my_loc\", \"precision\":\"13\"}");
GeoHashGridParser parser = new GeoHashGridParser();
try {
@ -81,4 +81,4 @@ public class GeoHashGridParserTests extends ESTestCase {
assertEquals("Invalid geohash aggregation precision of 13. Must be between 1 and 12.", ex.getMessage());
}
}
}
}

View File

@ -69,6 +69,11 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo;
*/
public class SignificanceHeuristicTests extends ESTestCase {
static class SignificantTermsTestSearchContext extends TestSearchContext {
public SignificantTermsTestSearchContext() {
super(null);
}
@Override
public int numberOfShards() {
return 1;

View File

@ -40,7 +40,7 @@ public class FieldDataFieldsTests extends ESTestCase {
parser.nextToken();
parser.nextToken();
parser.nextToken();
SearchContext context = new TestSearchContext();
SearchContext context = new TestSearchContext(null);
parseElement.parse(parser, context);
}
@ -52,7 +52,7 @@ public class FieldDataFieldsTests extends ESTestCase {
parser.nextToken();
parser.nextToken();
parser.nextToken();
SearchContext context = new TestSearchContext();
SearchContext context = new TestSearchContext(null);
parseElement.parse(parser, context);
}
@ -69,7 +69,7 @@ public class FieldDataFieldsTests extends ESTestCase {
parser.nextToken();
parser.nextToken();
parser.nextToken();
SearchContext context = new TestSearchContext();
SearchContext context = new TestSearchContext(null);
try {
parseElement.parse(parser, context);
fail("Expected IllegalStateException");

View File

@ -51,7 +51,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
public class QueryPhaseTests extends ESTestCase {
private void countTestCase(Query query, IndexReader reader, boolean shouldCollect) throws Exception {
TestSearchContext context = new TestSearchContext();
TestSearchContext context = new TestSearchContext(null);
context.parsedQuery(new ParsedQuery(query));
context.setSize(0);
@ -120,7 +120,7 @@ public class QueryPhaseTests extends ESTestCase {
}
public void testPostFilterDisablesCountOptimization() throws Exception {
TestSearchContext context = new TestSearchContext();
TestSearchContext context = new TestSearchContext(null);
context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery()));
context.setSize(0);
@ -143,7 +143,7 @@ public class QueryPhaseTests extends ESTestCase {
}
public void testMinScoreDisablesCountOptimization() throws Exception {
TestSearchContext context = new TestSearchContext();
TestSearchContext context = new TestSearchContext(null);
context.parsedQuery(new ParsedQuery(new MatchAllDocsQuery()));
context.setSize(0);

View File

@ -36,7 +36,7 @@ public class SortParserTests extends ESSingleNodeTestCase {
mapping.startObject().startObject("type").startObject("properties").startObject("location").field("type", "geo_point").endObject().endObject().endObject().endObject();
IndexService indexService = createIndex("testidx", Settings.settingsBuilder().build(), "type", mapping);
TestSearchContext context = (TestSearchContext) createSearchContext(indexService);
context.setTypes("type");
context.getQueryShardContext().setTypes("type");
XContentBuilder sortBuilder = jsonBuilder();
sortBuilder.startObject();

View File

@ -60,7 +60,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase {
Collections.singletonMap(Murmur3FieldMapper.CONTENT_TYPE, new Murmur3FieldMapper.TypeParser()),
Collections.emptyMap());
parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
}
public void testDefaults() throws Exception {
@ -136,7 +136,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase {
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
indexService = createIndex("test_bwc", settings);
parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "murmur3")
@ -152,7 +152,7 @@ public class Murmur3FieldMapperTests extends ESSingleNodeTestCase {
Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.V_1_4_2.id).build();
indexService = createIndex("test_bwc", settings);
parser = new DocumentMapperParser(indexService.getIndexSettings(), indexService.mapperService(),
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
String mapping = XContentFactory.jsonBuilder().startObject().startObject("type")
.startObject("properties").startObject("field")
.field("type", "murmur3")

View File

@ -66,7 +66,7 @@ public class SizeMappingTests extends ESSingleNodeTestCase {
Map<String, MetadataFieldMapper.TypeParser> metadataMappers = new HashMap<>();
IndicesModule indices = new IndicesModule();
indices.registerMetadataMapper(SizeFieldMapper.NAME, new SizeFieldMapper.TypeParser());
mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), indices.getMapperRegistry(), indexService::getQueryShardContext);
mapperService = new MapperService(indexService.getIndexSettings(), indexService.analysisService(), indexService.similarityService(), indices.getMapperRegistry(), indexService::newQueryShardContext);
parser = mapperService.documentMapperParser();
}
@ -98,7 +98,7 @@ public class SizeMappingTests extends ESSingleNodeTestCase {
Collections.emptyMap(),
Collections.singletonMap(SizeFieldMapper.NAME, new SizeFieldMapper.TypeParser()));
parser = new DocumentMapperParser(indexService.getIndexSettings(), mapperService,
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::getQueryShardContext);
indexService.analysisService(), indexService.similarityService(), mapperRegistry, indexService::newQueryShardContext);
DocumentMapper docMapper = parser.parse("type", new CompressedXContent(mapping));
BytesReference source = XContentFactory.jsonBuilder()

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.test;
import com.carrotsearch.hppc.ObjectObjectAssociativeContainer;
import org.apache.lucene.search.Collector;
import org.apache.lucene.search.FieldDoc;
import org.apache.lucene.search.Query;
@ -27,7 +26,6 @@ import org.apache.lucene.util.Counter;
import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cache.recycler.PageCacheRecycler;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.collect.ImmutableOpenMap;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.analysis.AnalysisService;
@ -38,6 +36,7 @@ import org.elasticsearch.index.mapper.MappedFieldType;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import org.elasticsearch.index.query.ParsedQuery;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.script.ScriptService;
@ -47,7 +46,6 @@ import org.elasticsearch.search.dfs.DfsSearchResult;
import org.elasticsearch.search.fetch.FetchSearchResult;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.FetchSubPhaseContext;
import org.elasticsearch.search.fetch.innerhits.InnerHitsContext;
import org.elasticsearch.search.fetch.script.ScriptFieldsContext;
import org.elasticsearch.search.fetch.source.FetchSourceContext;
import org.elasticsearch.search.highlight.SearchContextHighlight;
@ -62,11 +60,9 @@ import org.elasticsearch.search.rescore.RescoreSearchContext;
import org.elasticsearch.search.suggest.SuggestionSearchContext;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class TestSearchContext extends SearchContext {
@ -80,6 +76,7 @@ public class TestSearchContext extends SearchContext {
final IndexShard indexShard;
final Counter timeEstimateCounter = Counter.newCounter();
final QuerySearchResult queryResult = new QuerySearchResult();
final QueryShardContext queryShardContext;
ScriptService scriptService;
ParsedQuery originalQuery;
ParsedQuery postFilter;
@ -89,7 +86,6 @@ public class TestSearchContext extends SearchContext {
ContextIndexSearcher searcher;
int size;
private int terminateAfter = DEFAULT_TERMINATE_AFTER;
private String[] types;
private SearchContextAggregations aggregations;
private final long originNanoTime = System.nanoTime();
@ -105,9 +101,10 @@ public class TestSearchContext extends SearchContext {
this.threadPool = threadPool;
this.indexShard = indexService.getShardOrNull(0);
this.scriptService = scriptService;
queryShardContext = indexService.newQueryShardContext();
}
public TestSearchContext() {
public TestSearchContext(QueryShardContext queryShardContext) {
super(ParseFieldMatcher.STRICT);
this.pageCacheRecycler = null;
this.bigArrays = null;
@ -117,10 +114,7 @@ public class TestSearchContext extends SearchContext {
this.fixedBitSetFilterCache = null;
this.indexShard = null;
scriptService = null;
}
public void setTypes(String... types) {
this.types = types;
this.queryShardContext = queryShardContext;
}
@Override
@ -167,16 +161,6 @@ public class TestSearchContext extends SearchContext {
return 1;
}
@Override
public boolean hasTypes() {
return false;
}
@Override
public String[] types() {
return new String[0];
}
@Override
public float queryBoost() {
return 0;
@ -590,4 +574,9 @@ public class TestSearchContext extends SearchContext {
@Override
public Map<Class<?>, Collector> queryCollectors() {return queryCollectors;}
@Override
public QueryShardContext getQueryShardContext() {
return queryShardContext;
}
}