Add more context to QueryShardContext (#46584)

This change adds an IndexSearcher and the node's BigArrays in the QueryShardContext.
It's a spin off of #46527 as this change is required to allow aggregation builder to solely use the
query shard context.

Relates #46523
This commit is contained in:
Jim Ferenczi 2019-09-11 11:55:41 +02:00 committed by jimczi
parent 579af626f5
commit 425b1a77e8
32 changed files with 160 additions and 129 deletions

View File

@ -571,8 +571,10 @@ public class PainlessExecuteAction extends ActionType<PainlessExecuteAction.Resp
indexWriter.addDocuments(parsedDocument.docs());
try (IndexReader indexReader = DirectoryReader.open(indexWriter)) {
final long absoluteStartMillis = System.currentTimeMillis();
final IndexSearcher searcher = new IndexSearcher(indexReader);
searcher.setQueryCache(null);
QueryShardContext context =
indexService.newQueryShardContext(0, indexReader, () -> absoluteStartMillis, null);
indexService.newQueryShardContext(0, searcher, () -> absoluteStartMillis, null);
return handler.apply(context, indexReader.leaves().get(0));
}
}

View File

@ -258,7 +258,7 @@ public class PercolatorQuerySearchTests extends ESSingleNodeTestCase {
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
long[] currentTime = new long[] {System.currentTimeMillis()};
QueryShardContext queryShardContext =
indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> currentTime[0], null);
indexService.newQueryShardContext(0, searcher, () -> currentTime[0], null);
BytesReference source = BytesReference.bytes(jsonBuilder().startObject()
.field("field1", "value")

View File

@ -22,7 +22,6 @@ package org.elasticsearch.index;
import org.apache.logging.log4j.message.ParameterizedMessage;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Sort;
import org.apache.lucene.store.AlreadyClosedException;
@ -523,25 +522,17 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust
return indexSettings;
}
private IndexSearcher newCachedSearcher(int shardId, IndexReaderContext context) {
IndexSearcher searcher = new IndexSearcher(context);
searcher.setQueryCache(cache().query());
searcher.setQueryCachingPolicy(getShard(shardId).getQueryCachingPolicy());
return searcher;
}
/**
* Creates a new QueryShardContext. The context has not types set yet, if types are required set them via
* {@link QueryShardContext#setTypes(String...)}.
*
* Passing a {@code null} {@link IndexReader} will return a valid context, however it won't be able to make
* Passing a {@code null} {@link IndexSearcher} will return a valid context, however it won't be able to make
* {@link IndexReader}-specific optimizations, such as rewriting containing range queries.
*/
public QueryShardContext newQueryShardContext(int shardId, IndexReader indexReader, LongSupplier nowInMillis, String clusterAlias) {
public QueryShardContext newQueryShardContext(int shardId, IndexSearcher searcher, LongSupplier nowInMillis, String clusterAlias) {
return new QueryShardContext(
shardId, indexSettings, indexCache.bitsetFilterCache(), context -> newCachedSearcher(shardId, context),
indexFieldData::getForField, mapperService(), similarityService(), scriptService, xContentRegistry, namedWriteableRegistry,
client, indexReader, nowInMillis, clusterAlias);
shardId, indexSettings, bigArrays, indexCache.bitsetFilterCache(), indexFieldData::getForField, mapperService(),
similarityService(), scriptService, xContentRegistry, namedWriteableRegistry, client, searcher, nowInMillis, clusterAlias);
}
/**

View File

@ -20,9 +20,7 @@
package org.elasticsearch.index.fielddata;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.ReaderUtil;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.FieldComparatorSource;
@ -47,7 +45,6 @@ import org.elasticsearch.search.MultiValueMode;
import org.elasticsearch.search.sort.NestedSortBuilder;
import java.io.IOException;
import java.util.function.Function;
/**
* Thread-safe utility class that allows to get per-segment values via the
@ -115,24 +112,19 @@ public interface IndexFieldData<FD extends AtomicFieldData> extends IndexCompone
private final BitSetProducer rootFilter;
private final Query innerQuery;
private final NestedSortBuilder nestedSort;
private final Function<IndexReaderContext, IndexSearcher> searcherFactory;
private final IndexSearcher searcher;
public Nested(BitSetProducer rootFilter, Query innerQuery, NestedSortBuilder nestedSort,
Function<IndexReaderContext, IndexSearcher> searcherFactory) {
public Nested(BitSetProducer rootFilter, Query innerQuery, NestedSortBuilder nestedSort, IndexSearcher searcher) {
this.rootFilter = rootFilter;
this.innerQuery = innerQuery;
this.nestedSort = nestedSort;
this.searcherFactory = searcherFactory;
this.searcher = searcher;
}
public Query getInnerQuery() {
return innerQuery;
}
public BitSetProducer getRootFilter() {
return rootFilter;
}
public NestedSortBuilder getNestedSort() { return nestedSort; }
/**
@ -146,9 +138,7 @@ public interface IndexFieldData<FD extends AtomicFieldData> extends IndexCompone
* Get a {@link DocIdSet} that matches the inner documents.
*/
public DocIdSetIterator innerDocs(LeafReaderContext ctx) throws IOException {
final IndexReaderContext topLevelCtx = ReaderUtil.getTopLevelContext(ctx);
IndexSearcher indexSearcher = searcherFactory.apply(topLevelCtx);
Weight weight = indexSearcher.createWeight(indexSearcher.rewrite(innerQuery), ScoreMode.COMPLETE_NO_SCORES, 1f);
Weight weight = searcher.createWeight(searcher.rewrite(innerQuery), ScoreMode.COMPLETE_NO_SCORES, 1f);
Scorer s = weight.scorer(ctx);
return s == null ? null : s.iterator();
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.index.query;
import org.apache.logging.log4j.LogManager;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.join.BitSetProducer;
@ -37,6 +36,7 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
@ -68,7 +68,6 @@ import java.util.Map;
import java.util.Set;
import java.util.function.BiConsumer;
import java.util.function.BiFunction;
import java.util.function.Function;
import java.util.function.LongSupplier;
import static java.util.Collections.unmodifiableMap;
@ -84,13 +83,13 @@ public class QueryShardContext extends QueryRewriteContext {
private final ScriptService scriptService;
private final IndexSettings indexSettings;
private final BigArrays bigArrays;
private final MapperService mapperService;
private final SimilarityService similarityService;
private final BitsetFilterCache bitsetFilterCache;
private final Function<IndexReaderContext, IndexSearcher> searcherFactory;
private final BiFunction<MappedFieldType, String, IndexFieldData<?>> indexFieldDataService;
private final int shardId;
private final IndexReader reader;
private final IndexSearcher searcher;
private String[] types = Strings.EMPTY_ARRAY;
private boolean cacheable = true;
private final SetOnce<Boolean> frozen = new SetOnce<>();
@ -109,42 +108,58 @@ public class QueryShardContext extends QueryRewriteContext {
private boolean mapUnmappedFieldAsString;
private NestedScope nestedScope;
public QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache,
Function<IndexReaderContext, IndexSearcher> searcherFactory,
BiFunction<MappedFieldType, String, IndexFieldData<?>> indexFieldDataLookup, MapperService mapperService,
SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry,
NamedWriteableRegistry namedWriteableRegistry, Client client, IndexReader reader, LongSupplier nowInMillis,
String clusterAlias) {
this(shardId, indexSettings, bitsetFilterCache, searcherFactory, indexFieldDataLookup, mapperService, similarityService,
scriptService, xContentRegistry, namedWriteableRegistry, client, reader, nowInMillis,
public QueryShardContext(int shardId,
IndexSettings indexSettings,
BigArrays bigArrays,
BitsetFilterCache bitsetFilterCache,
BiFunction<MappedFieldType, String, IndexFieldData<?>> indexFieldDataLookup,
MapperService mapperService,
SimilarityService similarityService,
ScriptService scriptService,
NamedXContentRegistry xContentRegistry,
NamedWriteableRegistry namedWriteableRegistry,
Client client,
IndexSearcher searcher,
LongSupplier nowInMillis,
String clusterAlias) {
this(shardId, indexSettings, bigArrays, bitsetFilterCache, indexFieldDataLookup, mapperService, similarityService,
scriptService, xContentRegistry, namedWriteableRegistry, client, searcher, nowInMillis,
new Index(RemoteClusterAware.buildRemoteIndexName(clusterAlias, indexSettings.getIndex().getName()),
indexSettings.getIndex().getUUID()));
}
public QueryShardContext(QueryShardContext source) {
this(source.shardId, source.indexSettings, source.bitsetFilterCache, source.searcherFactory, source.indexFieldDataService,
this(source.shardId, source.indexSettings, source.bigArrays, source.bitsetFilterCache, source.indexFieldDataService,
source.mapperService, source.similarityService, source.scriptService, source.getXContentRegistry(),
source.getWriteableRegistry(), source.client, source.reader, source.nowInMillis, source.fullyQualifiedIndex);
source.getWriteableRegistry(), source.client, source.searcher, source.nowInMillis, source.fullyQualifiedIndex);
}
private QueryShardContext(int shardId, IndexSettings indexSettings, BitsetFilterCache bitsetFilterCache,
Function<IndexReaderContext, IndexSearcher> searcherFactory,
BiFunction<MappedFieldType, String, IndexFieldData<?>> indexFieldDataLookup, MapperService mapperService,
SimilarityService similarityService, ScriptService scriptService, NamedXContentRegistry xContentRegistry,
NamedWriteableRegistry namedWriteableRegistry, Client client, IndexReader reader, LongSupplier nowInMillis,
Index fullyQualifiedIndex) {
private QueryShardContext(int shardId,
IndexSettings indexSettings,
BigArrays bigArrays,
BitsetFilterCache bitsetFilterCache,
BiFunction<MappedFieldType, String, IndexFieldData<?>> indexFieldDataLookup,
MapperService mapperService,
SimilarityService similarityService,
ScriptService scriptService,
NamedXContentRegistry xContentRegistry,
NamedWriteableRegistry namedWriteableRegistry,
Client client,
IndexSearcher searcher,
LongSupplier nowInMillis,
Index fullyQualifiedIndex) {
super(xContentRegistry, namedWriteableRegistry, client, nowInMillis);
this.shardId = shardId;
this.similarityService = similarityService;
this.mapperService = mapperService;
this.bitsetFilterCache = bitsetFilterCache;
this.searcherFactory = searcherFactory;
this.indexFieldDataService = indexFieldDataLookup;
this.allowUnmappedFields = indexSettings.isDefaultAllowUnmappedFields();
this.nestedScope = new NestedScope();
this.scriptService = scriptService;
this.indexSettings = indexSettings;
this.reader = reader;
this.bigArrays = bigArrays;
this.searcher = searcher;
this.fullyQualifiedIndex = fullyQualifiedIndex;
}
@ -183,10 +198,6 @@ public class QueryShardContext extends QueryRewriteContext {
return bitsetFilterCache.getBitSetProducer(filter);
}
public IndexSearcher newCachedSearcher(IndexReaderContext context) {
return searcherFactory.apply(context);
}
public <IFD extends IndexFieldData<?>> IFD getForField(MappedFieldType fieldType) {
return (IFD) indexFieldDataService.apply(fieldType, fullyQualifiedIndex.getName());
}
@ -424,7 +435,13 @@ public class QueryShardContext extends QueryRewriteContext {
/** Return the current {@link IndexReader}, or {@code null} if no index reader is available,
* for instance if this rewrite context is used to index queries (percolation). */
public IndexReader getIndexReader() {
return reader;
return searcher != null ? searcher.getIndexReader() : null;
}
/** Return the current {@link IndexSearcher}, or {@code null} if no index reader is available,
* for instance if this rewrite context is used to index queries (percolation). */
public IndexSearcher searcher() {
return searcher;
}
/**
@ -433,4 +450,11 @@ public class QueryShardContext extends QueryRewriteContext {
public Index getFullyQualifiedIndex() {
return fullyQualifiedIndex;
}
/**
* Return the {@link BigArrays} instance for this node.
*/
public BigArrays bigArrays() {
return bigArrays;
}
}

View File

@ -180,7 +180,7 @@ final class DefaultSearchContext extends SearchContext {
this.relativeTimeSupplier = relativeTimeSupplier;
this.timeout = timeout;
this.minNodeVersion = minNodeVersion;
queryShardContext = indexService.newQueryShardContext(request.shardId().id(), searcher.getIndexReader(), request::nowInMillis,
queryShardContext = indexService.newQueryShardContext(request.shardId().id(), searcher, request::nowInMillis,
shardTarget.getClusterAlias());
queryShardContext.setTypes(request.types());
queryBoost = request.indexBoost();

View File

@ -195,7 +195,7 @@ public abstract class SortBuilder<T extends SortBuilder<T>> implements NamedWrit
} else {
parentQuery = objectMapper.nestedTypeFilter();
}
return new Nested(context.bitsetFilter(parentQuery), childQuery, nestedSort, context::newCachedSearcher);
return new Nested(context.bitsetFilter(parentQuery), childQuery, nestedSort, context.searcher());
}
private static Query resolveNestedQuery(QueryShardContext context, NestedSortBuilder nestedSort, Query parentQuery) throws IOException {

View File

@ -163,7 +163,7 @@ public abstract class AbstractFieldDataTestCase extends ESSingleNodeTestCase {
protected Nested createNested(IndexSearcher searcher, Query parentFilter, Query childFilter) throws IOException {
BitsetFilterCache s = indexService.cache().bitsetFilterCache();
return new Nested(s.getBitSetProducer(parentFilter), childFilter, null, IndexSearcher::new);
return new Nested(s.getBitSetProducer(parentFilter), childFilter, null, searcher);
}
public void testEmpty() throws Exception {

View File

@ -38,6 +38,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.time.DateFormatter;
import org.elasticsearch.common.time.DateFormatters;
import org.elasticsearch.common.time.DateMathParser;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.AtomicNumericFieldData;
@ -176,9 +177,9 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
Settings indexSettings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).build();
QueryShardContext context = new QueryShardContext(0,
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(),
indexSettings),
null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null);
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings),
BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null,
xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null);
MappedFieldType ft = createDefaultFieldType();
ft.setName("field");
String date = "2015-10-12T14:10:55";
@ -200,7 +201,8 @@ public class DateFieldTypeTests extends FieldTypeTestCase {
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).build();
QueryShardContext context = new QueryShardContext(0,
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings),
null, null, null, null, null, null, xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null);
BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(),
null, null, () -> nowInMillis, null);
MappedFieldType ft = createDefaultFieldType();
ft.setName("field");
String date1 = "2015-10-12T14:10:55";

View File

@ -24,6 +24,7 @@ import org.apache.lucene.search.TermQuery;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.query.QueryShardContext;
import org.junit.Before;
@ -66,7 +67,8 @@ public class FieldNamesFieldTypeTests extends FieldTypeTestCase {
when(mapperService.simpleMatchToFullName("field_name")).thenReturn(Collections.singleton("field_name"));
QueryShardContext queryShardContext = new QueryShardContext(0,
indexSettings, null, null, null, mapperService, null, null, null, null, null, null, () -> 0L, null);
indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService,
null, null, null, null, null, null, () -> 0L, null);
fieldNamesFieldType.setEnabled(true);
Query termQuery = fieldNamesFieldType.termQuery("field_name", queryShardContext);
assertEquals(new TermQuery(new Term(FieldNamesFieldMapper.CONTENT_TYPE, "field_name")), termQuery);

View File

@ -38,6 +38,7 @@ import org.elasticsearch.common.geo.ShapeRelation;
import org.elasticsearch.common.network.InetAddresses;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.time.DateFormatter;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.RangeFieldMapper.RangeFieldType;
import org.elasticsearch.index.query.QueryShardContext;
@ -227,8 +228,8 @@ public class RangeFieldTypeTests extends FieldTypeTestCase {
Settings indexSettings = Settings.builder()
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings);
return new QueryShardContext(0, idxSettings, null, null, null, null, null, null, xContentRegistry(),
writableRegistry(), null, null, () -> nowInMillis, null);
return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null,
xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null);
}
public void testDateRangeQueryUsingMappingFormat() {

View File

@ -29,6 +29,7 @@ import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.script.Script;
@ -370,9 +371,8 @@ public class IntervalQueryBuilderTests extends AbstractQueryTestCase<IntervalQue
QueryShardContext baseContext = createShardContext();
QueryShardContext context = new QueryShardContext(baseContext.getShardId(), baseContext.getIndexSettings(),
null, null, null, baseContext.getMapperService(), null,
scriptService,
null, null, null, null, null, null);
BigArrays.NON_RECYCLING_INSTANCE, null, null, baseContext.getMapperService(),
null, scriptService, null, null, null, null, null, null);
String json = "{ \"intervals\" : { \"" + STRING_FIELD_NAME + "\": { " +
"\"match\" : { " +

View File

@ -25,6 +25,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.fielddata.IndexFieldData;
@ -128,9 +129,10 @@ public class QueryShardContextTests extends ESTestCase {
final long nowInMillis = randomNonNegativeLong();
return new QueryShardContext(
0, indexSettings, null, null, (mappedFieldType, idxName) ->
mappedFieldType.fielddataBuilder(idxName).build(indexSettings, mappedFieldType, null, null, null)
, mapperService, null, null, NamedXContentRegistry.EMPTY, new NamedWriteableRegistry(Collections.emptyList()), null, null,
() -> nowInMillis, clusterAlias);
0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null,
(mappedFieldType, idxName) ->
mappedFieldType.fielddataBuilder(idxName).build(indexSettings, mappedFieldType, null, null, null),
mapperService, null, null, NamedXContentRegistry.EMPTY, new NamedWriteableRegistry(Collections.emptyList()),
null, null, () -> nowInMillis, clusterAlias);
}
}

View File

@ -21,8 +21,10 @@ package org.elasticsearch.index.query;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.MultiReader;
import org.apache.lucene.search.IndexSearcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.compress.CompressedXContent;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.mapper.MappedFieldType.Relation;
@ -37,8 +39,9 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase {
public void testRewriteMissingField() throws Exception {
IndexService indexService = createIndex("test");
IndexReader reader = new MultiReader();
QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, null,
indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(), null, reader, null, null);
QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE,
null, null, indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(),
null, new IndexSearcher(reader), null, null);
RangeQueryBuilder range = new RangeQueryBuilder("foo");
assertEquals(Relation.DISJOINT, range.getRelation(context));
}
@ -73,8 +76,9 @@ public class RangeQueryRewriteTests extends ESSingleNodeTestCase {
indexService.mapperService().merge("type",
new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE);
IndexReader reader = new MultiReader();
QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, null,
indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(), null, reader, null, null);
QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE,
null, null, indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(),
null, new IndexSearcher(reader), null, null);
RangeQueryBuilder range = new RangeQueryBuilder("foo");
// no values -> DISJOINT
assertEquals(Relation.DISJOINT, range.getRelation(context));

View File

@ -28,6 +28,7 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.queries.SpanMatchNoDocsQuery;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query;
@ -274,7 +275,7 @@ public class SpanMultiTermQueryBuilderTests extends AbstractQueryTestCase<SpanMu
QueryBuilder queryBuilder = new SpanMultiTermQueryBuilder(
QueryBuilders.prefixQuery("body", "bar")
);
Query query = queryBuilder.toQuery(createShardContext(reader));
Query query = queryBuilder.toQuery(createShardContext(new IndexSearcher(reader)));
RuntimeException exc = expectThrows(RuntimeException.class, () -> query.rewrite(reader));
assertThat(exc.getMessage(), containsString("maxClauseCount"));
} finally {

View File

@ -20,8 +20,6 @@
package org.elasticsearch.index.query.plugin;
import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Before;
@ -64,10 +62,4 @@ public class CustomQueryParserIT extends ESIntegTestCase {
public void testCustomDummyQueryWithinBooleanQuery() {
assertHitCount(client().prepareSearch("index").setQuery(new BoolQueryBuilder().must(new DummyQueryBuilder())).get(), 1L);
}
private static QueryShardContext queryShardContext() {
IndicesService indicesService = internalCluster().getDataNodeInstance(IndicesService.class);
return indicesService.indexServiceSafe(resolveIndex("index")).newQueryShardContext(
randomInt(20), null, () -> { throw new UnsupportedOperationException(); }, null);
}
}

View File

@ -23,6 +23,7 @@ import org.apache.lucene.index.MultiReader;
import org.apache.lucene.index.Term;
import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.search.MatchNoDocsQuery;
import org.apache.lucene.search.Query;
@ -327,7 +328,7 @@ public class NestedHelperTests extends ESSingleNodeTestCase {
}
public void testNested() throws IOException {
QueryShardContext context = indexService.newQueryShardContext(0, new MultiReader(), () -> 0, null);
QueryShardContext context = indexService.newQueryShardContext(0, new IndexSearcher(new MultiReader()), () -> 0, null);
NestedQueryBuilder queryBuilder = new NestedQueryBuilder("nested1", new MatchAllQueryBuilder(), ScoreMode.Avg);
ESToParentBlockJoinQuery query = (ESToParentBlockJoinQuery) queryBuilder.toQuery(context);

View File

@ -613,7 +613,7 @@ public class NestedSortingTests extends AbstractFieldDataTestCase {
DirectoryReader reader = DirectoryReader.open(writer);
reader = ElasticsearchDirectoryReader.wrap(reader, new ShardId(indexService.index(), 0));
IndexSearcher searcher = new IndexSearcher(reader);
QueryShardContext queryShardContext = indexService.newQueryShardContext(0, reader, () -> 0L, null);
QueryShardContext queryShardContext = indexService.newQueryShardContext(0, searcher, () -> 0L, null);
FieldSortBuilder sortBuilder = new FieldSortBuilder("chapters.paragraphs.word_count");
sortBuilder.setNestedSort(new NestedSortBuilder("chapters").setNestedSort(new NestedSortBuilder("chapters.paragraphs")));

View File

@ -25,6 +25,7 @@ import org.apache.lucene.document.SortedNumericDocValuesField;
import org.apache.lucene.index.DirectoryReader;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.store.Directory;
import org.elasticsearch.common.time.DateFormatter;
import org.elasticsearch.common.time.DateFormatters;
@ -152,8 +153,8 @@ public class DateHistogramTests extends BaseAggregationTestCase<DateHistogramAgg
try (IndexReader readerThatCrosses = DirectoryReader.open(w)) {
QueryShardContext shardContextThatDoesntCross = createShardContext(readerThatDoesntCross);
QueryShardContext shardContextThatCrosses = createShardContext(readerThatCrosses);
QueryShardContext shardContextThatDoesntCross = createShardContext(new IndexSearcher(readerThatDoesntCross));
QueryShardContext shardContextThatCrosses = createShardContext(new IndexSearcher(readerThatCrosses));
DateHistogramAggregationBuilder builder = new DateHistogramAggregationBuilder("my_date_histo");
builder.field(DATE_FIELD_NAME);

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.time.DateFormatter;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
@ -99,8 +100,9 @@ public class ExtendedBoundsTests extends ESTestCase {
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1).put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 1).build();
SearchContext context = mock(SearchContext.class);
QueryShardContext qsc = new QueryShardContext(0,
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), null, null, null, null,
null, null, xContentRegistry(), writableRegistry(), null, null, () -> now, null);
new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings),
BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(),
null, null, () -> now, null);
when(context.getQueryShardContext()).thenReturn(qsc);
DateFormatter formatter = DateFormatter.forPattern("dateOptionalTime");
DocValueFormat format = new DocValueFormat.DateTime(formatter, ZoneOffset.UTC, DateFieldMapper.Resolution.MILLISECONDS);

View File

@ -26,6 +26,7 @@ import org.apache.lucene.index.RandomIndexWriter;
import org.apache.lucene.search.MatchAllDocsQuery;
import org.apache.lucene.store.Directory;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.QueryShardContext;
@ -421,7 +422,7 @@ public class ScriptedMetricAggregatorTests extends AggregatorTestCase {
MockScriptEngine scriptEngine = new MockScriptEngine(MockScriptEngine.NAME, SCRIPTS, Collections.emptyMap());
Map<String, ScriptEngine> engines = Collections.singletonMap(scriptEngine.getType(), scriptEngine);
ScriptService scriptService = new ScriptService(Settings.EMPTY, engines, ScriptModule.CORE_CONTEXTS);
return new QueryShardContext(0, indexSettings, null, null, null, mapperService, null, scriptService,
return new QueryShardContext(0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, null, scriptService,
xContentRegistry(), writableRegistry(), null, null, System::currentTimeMillis, null);
}
}

View File

@ -42,7 +42,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Bytes> config = ValuesSourceConfig.resolve(
context, null, "bytes", null, null, null, null);
@ -64,7 +64,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Bytes> config = ValuesSourceConfig.resolve(
context, null, "bytes", null, null, null, null);
@ -91,7 +91,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Bytes> config = ValuesSourceConfig.resolve(
context, ValueType.STRING, "bytes", null, null, null, null);
ValuesSource.Bytes valuesSource = config.toValuesSource(context);
@ -117,7 +117,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Numeric> config = ValuesSourceConfig.resolve(
context, null, "long", null, null, null, null);
@ -139,7 +139,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Numeric> config = ValuesSourceConfig.resolve(
context, null, "long", null, null, null, null);
@ -166,7 +166,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Numeric> config = ValuesSourceConfig.resolve(
context, ValueType.NUMBER, "long", null, null, null, null);
@ -193,7 +193,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Numeric> config = ValuesSourceConfig.resolve(
context, null, "bool", null, null, null, null);
@ -215,7 +215,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Numeric> config = ValuesSourceConfig.resolve(
context, null, "bool", null, null, null, null);
@ -242,7 +242,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Numeric> config = ValuesSourceConfig.resolve(
context, ValueType.BOOLEAN, "bool", null, null, null, null);
@ -263,7 +263,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
public void testTypeFieldDeprecation() {
IndexService indexService = createIndex("index", Settings.EMPTY, "type");
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Bytes> config = ValuesSourceConfig.resolve(
context, null, TypeFieldMapper.NAME, null, null, null, null);
@ -280,7 +280,7 @@ public class ValuesSourceConfigTests extends ESSingleNodeTestCase {
.get();
try (Engine.Searcher searcher = indexService.getShard(0).acquireSearcher("test")) {
QueryShardContext context = indexService.newQueryShardContext(0, searcher.getIndexReader(), () -> 42L, null);
QueryShardContext context = indexService.newQueryShardContext(0, searcher, () -> 42L, null);
ValuesSourceConfig<ValuesSource.Bytes> config = ValuesSourceConfig.resolve(
context, ValueType.STRING, "alias", null, null, null, null);
ValuesSource.Bytes valuesSource = config.toValuesSource(context);

View File

@ -27,6 +27,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -277,8 +278,9 @@ public class HighlightBuilderTests extends ESTestCase {
Index index = new Index(randomAlphaOfLengthBetween(1, 10), "_na_");
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(index, indexSettings);
// shard context will only need indicesQueriesRegistry for building Query objects nested in highlighter
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, null,
xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null) {
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE,
null, null, null, null, null, xContentRegistry(), namedWriteableRegistry,
null, null, System::currentTimeMillis, null) {
@Override
public MappedFieldType fieldMapper(String name) {
TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name);

View File

@ -26,6 +26,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedObjectNotFoundException;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
@ -141,7 +142,8 @@ public class QueryRescorerBuilderTests extends ESTestCase {
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build();
IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings);
// shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, null, null, null,
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE,
null, null, null, null, null,
xContentRegistry(), namedWriteableRegistry, null, null, () -> nowInMillis, null) {
@Override
public MappedFieldType fieldMapper(String name) {

View File

@ -19,12 +19,12 @@
package org.elasticsearch.search.sort;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.SortField;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -191,8 +191,8 @@ public abstract class AbstractSortTestCase<T extends SortBuilder<T>> extends EST
IndexFieldData.Builder builder = fieldType.fielddataBuilder(fieldIndexName);
return builder.build(idxSettings, fieldType, new IndexFieldDataCache.None(), null, null);
};
return new QueryShardContext(0, idxSettings, bitsetFilterCache, IndexSearcher::new, indexFieldDataLookup, null, null,
scriptService, xContentRegistry(), namedWriteableRegistry, null, null, () -> randomNonNegativeLong(), null) {
return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache, indexFieldDataLookup,
null, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, () -> randomNonNegativeLong(), null) {
@Override
public MappedFieldType fieldMapper(String name) {

View File

@ -25,6 +25,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
@ -178,8 +179,9 @@ public abstract class AbstractSuggestionBuilderTestCase<SB extends SuggestionBui
invocation -> new NamedAnalyzer((String) invocation.getArguments()[0], AnalyzerScope.INDEX, new SimpleAnalyzer()));
when(scriptService.compile(any(Script.class), any())).then(invocation -> new TestTemplateService.MockTemplateScript.Factory(
((Script) invocation.getArguments()[0]).getIdOrCode()));
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, null, null, null, mapperService, null,
scriptService, xContentRegistry(), namedWriteableRegistry, null, null, System::currentTimeMillis, null);
QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null,
null, mapperService, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null,
System::currentTimeMillis, null);
SuggestionContext suggestionContext = suggestionBuilder.build(mockShardContext);
assertEquals(toBytesRef(suggestionBuilder.text()), suggestionContext.getText());

View File

@ -39,6 +39,7 @@ import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.MockBigArrays;
import org.elasticsearch.common.util.MockPageCacheRecycler;
import org.elasticsearch.index.Index;
@ -300,7 +301,7 @@ public abstract class AggregatorTestCase extends ESTestCase {
protected QueryShardContext queryShardContextMock(MapperService mapperService, IndexSettings indexSettings,
CircuitBreakerService circuitBreakerService) {
return new QueryShardContext(0, indexSettings, null, null,
return new QueryShardContext(0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null,
getIndexFieldDataLookup(mapperService, circuitBreakerService),
mapperService, null, getMockScriptService(), xContentRegistry(),
writableRegistry(), null, null, System::currentTimeMillis, null);

View File

@ -21,7 +21,6 @@ package org.elasticsearch.test;
import com.carrotsearch.randomizedtesting.RandomizedTest;
import com.carrotsearch.randomizedtesting.SeedUtils;
import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.util.Accountable;
import org.elasticsearch.Version;
@ -41,6 +40,7 @@ import org.elasticsearch.common.settings.IndexScopedSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsModule;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.core.internal.io.IOUtils;
import org.elasticsearch.env.Environment;
@ -270,10 +270,10 @@ public abstract class AbstractBuilderTestCase extends ESTestCase {
}
/**
* @return a new {@link QueryShardContext} with the provided reader
* @return a new {@link QueryShardContext} with the provided searcher
*/
protected static QueryShardContext createShardContext(IndexReader reader) {
return serviceHolder.createShardContext(reader);
protected static QueryShardContext createShardContext(IndexSearcher searcher) {
return serviceHolder.createShardContext(searcher);
}
/**
@ -422,10 +422,10 @@ public abstract class AbstractBuilderTestCase extends ESTestCase {
public void close() throws IOException {
}
QueryShardContext createShardContext(IndexReader reader) {
return new QueryShardContext(0, idxSettings, bitsetFilterCache, IndexSearcher::new, indexFieldDataService::getForField,
mapperService, similarityService, scriptService, xContentRegistry, namedWriteableRegistry, this.client, reader,
() -> nowInMillis, null);
QueryShardContext createShardContext(IndexSearcher searcher) {
return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache,
indexFieldDataService::getForField, mapperService, similarityService, scriptService, xContentRegistry,
namedWriteableRegistry, this.client, searcher, () -> nowInMillis, null);
}
ScriptModule createScriptModule(List<ScriptPlugin> scriptPlugins) {

View File

@ -26,6 +26,7 @@ import org.elasticsearch.action.search.SearchType;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.lucene.search.Queries;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.shard.ShardId;
@ -41,7 +42,7 @@ public class MockSearchServiceTests extends ESTestCase {
public void testAssertNoInFlightContext() {
final long nowInMillis = randomNonNegativeLong();
SearchContext s = new TestSearchContext(new QueryShardContext(0,
new IndexSettings(EMPTY_INDEX_METADATA, Settings.EMPTY), null, null, null, null, null, null,
new IndexSettings(EMPTY_INDEX_METADATA, Settings.EMPTY), BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null,
xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null)) {
@Override

View File

@ -15,12 +15,14 @@ import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexWriterConfig;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.NoMergePolicy;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.Query;
import org.apache.lucene.store.Directory;
import org.apache.lucene.util.BitSet;
import org.elasticsearch.client.Client;
import org.elasticsearch.common.CheckedBiConsumer;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.query.QueryBuilders;
@ -237,8 +239,9 @@ public class DocumentSubsetBitsetCacheTests extends ESTestCase {
try (DirectoryReader directoryReader = DirectoryReader.open(directory)) {
final LeafReaderContext leaf = directoryReader.leaves().get(0);
final QueryShardContext context = new QueryShardContext(shardId.id(), indexSettings, null, null, null, mapperService,
null, null, xContentRegistry(), writableRegistry(), client, leaf.reader(), () -> nowInMillis, null);
final QueryShardContext context = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE,
null, null, mapperService, null, null, xContentRegistry(), writableRegistry(),
client, new IndexSearcher(directoryReader), () -> nowInMillis, null);
body.accept(context, leaf);
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.IndexSettings;
@ -82,8 +83,9 @@ public class SecurityIndexReaderWrapperIntegrationTests extends AbstractBuilderT
Client client = mock(Client.class);
when(client.settings()).thenReturn(Settings.EMPTY);
final long nowInMillis = randomNonNegativeLong();
QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, null, null, null, mapperService,
null, null, xContentRegistry(), writableRegistry(), client, null, () -> nowInMillis, null);
QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE,
null, null, mapperService, null, null, xContentRegistry(), writableRegistry(),
client, null, () -> nowInMillis, null);
QueryShardContext queryShardContext = spy(realQueryShardContext);
DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY);
XPackLicenseState licenseState = mock(XPackLicenseState.class);
@ -196,8 +198,9 @@ public class SecurityIndexReaderWrapperIntegrationTests extends AbstractBuilderT
Client client = mock(Client.class);
when(client.settings()).thenReturn(Settings.EMPTY);
final long nowInMillis = randomNonNegativeLong();
QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, null, null, null, mapperService,
null, null, xContentRegistry(), writableRegistry(), client, null, () -> nowInMillis, null);
QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE,
null, null, mapperService, null, null, xContentRegistry(), writableRegistry(),
client, null, () -> nowInMillis, null);
QueryShardContext queryShardContext = spy(realQueryShardContext);
DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY);

View File

@ -32,6 +32,7 @@ import org.elasticsearch.action.search.ShardSearchFailure;
import org.elasticsearch.common.Rounding;
import org.elasticsearch.common.time.DateFormatter;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.DateFieldMapper;
@ -91,7 +92,7 @@ public class RollupIndexerIndexingTests extends AggregatorTestCase {
private void setup() {
settings = createIndexSettings();
queryShardContext = new QueryShardContext(0, settings,
null, null, null, null, null, null,
BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null,
null, null, null, null, () -> 0L, null);
}