diff --git a/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/90_index_name_query.yml b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/90_index_name_query.yml new file mode 100644 index 0000000000000..030dad662df59 --- /dev/null +++ b/qa/multi-cluster-search/src/test/resources/rest-api-spec/test/multi_cluster/90_index_name_query.yml @@ -0,0 +1,58 @@ +--- +setup: + - do: + indices.create: + index: single_doc_index + body: + settings: + index: + number_of_shards: 1 + number_of_replicas: 0 +--- +teardown: + - do: + indices.delete: + index: single_doc_index + ignore_unavailable: true + +--- +"Test that queries on _index match against the correct indices.": + + - do: + bulk: + refresh: true + body: + - '{"index": {"_index": "single_doc_index"}}' + - '{"f1": "local_cluster", "sort_field": 0}' + + - do: + search: + rest_total_hits_as_int: true + index: "single_doc_index,my_remote_cluster:single_doc_index" + body: + query: + term: + "_index": "single_doc_index" + + - match: { hits.total: 1 } + - match: { hits.hits.0._index: "single_doc_index"} + - match: { _shards.total: 2 } + - match: { _shards.successful: 2 } + - match: { _shards.skipped : 0} + - match: { _shards.failed: 0 } + + - do: + search: + rest_total_hits_as_int: true + index: "single_doc_index,my_remote_cluster:single_doc_index" + body: + query: + term: + "_index": "my_remote_cluster:single_doc_index" + + - match: { hits.total: 1 } + - match: { hits.hits.0._index: "my_remote_cluster:single_doc_index"} + - match: { _shards.total: 2 } + - match: { _shards.successful: 2 } + - match: { _shards.skipped : 0} + - match: { _shards.failed: 0 } diff --git a/server/src/main/java/org/elasticsearch/index/IndexModule.java b/server/src/main/java/org/elasticsearch/index/IndexModule.java index 6ef335144eb52..b10d84ef1c627 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexModule.java +++ b/server/src/main/java/org/elasticsearch/index/IndexModule.java @@ -30,6 +30,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.TriFunction; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -386,6 +387,7 @@ public IndexService newIndexService( BigArrays bigArrays, ThreadPool threadPool, ScriptService scriptService, + ClusterService clusterService, Client client, IndicesQueryCache indicesQueryCache, MapperRegistry mapperRegistry, @@ -411,7 +413,7 @@ public IndexService newIndexService( return new IndexService(indexSettings, indexCreationContext, environment, xContentRegistry, new SimilarityService(indexSettings, scriptService, similarities), shardStoreDeleter, analysisRegistry, engineFactory, circuitBreakerService, bigArrays, threadPool, scriptService, - client, queryCache, directoryFactory, eventListener, readerWrapperFactory, mapperRegistry, + clusterService, client, queryCache, directoryFactory, eventListener, readerWrapperFactory, mapperRegistry, indicesFieldDataCache, searchOperationListeners, indexOperationListeners, namedWriteableRegistry); } diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 3f521bc67bd2d..ee53e35e1defa 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -31,6 +31,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -57,6 +58,7 @@ import org.elasticsearch.index.fielddata.IndexFieldDataCache; import org.elasticsearch.index.fielddata.IndexFieldDataService; import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.index.query.SearchIndexNameMatcher; import org.elasticsearch.index.query.QueryShardContext; import org.elasticsearch.index.seqno.RetentionLeaseSyncer; import org.elasticsearch.index.shard.IndexEventListener; @@ -133,6 +135,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final ThreadPool threadPool; private final BigArrays bigArrays; private final ScriptService scriptService; + private final ClusterService clusterService; private final Client client; private final CircuitBreakerService circuitBreakerService; private Supplier indexSortSupplier; @@ -150,6 +153,7 @@ public IndexService( BigArrays bigArrays, ThreadPool threadPool, ScriptService scriptService, + ClusterService clusterService, Client client, QueryCache queryCache, IndexStorePlugin.DirectoryFactory directoryFactory, @@ -200,6 +204,7 @@ public IndexService( this.bigArrays = bigArrays; this.threadPool = threadPool; this.scriptService = scriptService; + this.clusterService = clusterService; this.client = client; this.eventListener = eventListener; this.nodeEnv = nodeEnv; @@ -528,9 +533,11 @@ public IndexSettings getIndexSettings() { * {@link IndexReader}-specific optimizations, such as rewriting containing range queries. */ public QueryShardContext newQueryShardContext(int shardId, IndexSearcher searcher, LongSupplier nowInMillis, String clusterAlias) { + SearchIndexNameMatcher indexNameMatcher = new SearchIndexNameMatcher(index().getName(), clusterAlias, clusterService); return new QueryShardContext( shardId, indexSettings, bigArrays, indexCache.bitsetFilterCache(), indexFieldData::getForField, mapperService(), - similarityService(), scriptService, xContentRegistry, namedWriteableRegistry, client, searcher, nowInMillis, clusterAlias); + similarityService(), scriptService, xContentRegistry, namedWriteableRegistry, client, searcher, nowInMillis, clusterAlias, + indexNameMatcher); } /** diff --git a/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java b/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java index 276a8e7583c0e..4e6906401351d 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/IndexFieldMapper.java @@ -129,11 +129,16 @@ public Query existsQuery(QueryShardContext context) { */ @Override public Query termQuery(Object value, @Nullable QueryShardContext context) { - if (isSameIndex(value, context.getFullyQualifiedIndex().getName())) { + String pattern = value instanceof BytesRef + ? ((BytesRef) value).utf8ToString() + : value.toString(); + if (context.indexMatches(pattern)) { + // No need to OR these clauses - we can only logically be + // running in the context of just one of these index names. return Queries.newMatchAllQuery(); } else { - return Queries.newMatchNoDocsQuery("Index didn't match. Index queried: " + context.index().getName() - + " vs. " + value); + return Queries.newMatchNoDocsQuery("The index [" + context.getFullyQualifiedIndex().getName() + + "] doesn't match the provided value [" + value + "]."); } } @@ -143,26 +148,29 @@ public Query termsQuery(List values, QueryShardContext context) { return super.termsQuery(values, context); } for (Object value : values) { - if (isSameIndex(value, context.getFullyQualifiedIndex().getName())) { + String pattern = value instanceof BytesRef + ? ((BytesRef) value).utf8ToString() + : value.toString(); + if (context.indexMatches(pattern)) { // No need to OR these clauses - we can only logically be // running in the context of just one of these index names. return Queries.newMatchAllQuery(); } } // None of the listed index names are this one - return Queries.newMatchNoDocsQuery("Index didn't match. Index queried: " + context.getFullyQualifiedIndex().getName() - + " vs. " + values); + return Queries.newMatchNoDocsQuery("The index [" + context.getFullyQualifiedIndex().getName() + + "] doesn't match the provided values [" + values + "]."); } @Override public Query prefixQuery(String value, @Nullable MultiTermQuery.RewriteMethod method, QueryShardContext context) { - String indexName = context.getFullyQualifiedIndex().getName(); - if (indexName.startsWith(value)) { + String pattern = value + "*"; + if (context.indexMatches(pattern)) { return Queries.newMatchAllQuery(); } else { - return Queries.newMatchNoDocsQuery("The index [" + indexName + + return Queries.newMatchNoDocsQuery("The index [" + context.getFullyQualifiedIndex().getName() + "] doesn't match the provided prefix [" + value + "]."); } } @@ -176,8 +184,8 @@ public Query regexpQuery(String value, int flags, int maxDeterminizedStates, if (pattern.matcher(indexName).matches()) { return Queries.newMatchAllQuery(); } else { - return Queries.newMatchNoDocsQuery("The index [" + indexName + - "] doesn't match the provided pattern [" + value + "]."); + return Queries.newMatchNoDocsQuery("The index [" + context.getFullyQualifiedIndex().getName() + + "] doesn't match the provided pattern [" + value + "]."); } } @@ -185,20 +193,14 @@ public Query regexpQuery(String value, int flags, int maxDeterminizedStates, public Query wildcardQuery(String value, @Nullable MultiTermQuery.RewriteMethod method, QueryShardContext context) { - String indexName = context.getFullyQualifiedIndex().getName(); - if (isSameIndex(value, indexName)) { + if (context.indexMatches(value)) { return Queries.newMatchAllQuery(); } else { - return Queries.newMatchNoDocsQuery("The index [" + indexName + - "] doesn't match the provided pattern [" + value + "]."); + return Queries.newMatchNoDocsQuery("The index [" + context.getFullyQualifiedIndex().getName() + + "] doesn't match the provided pattern [" + value + "]."); } } - private boolean isSameIndex(Object value, String indexName) { - String pattern = value instanceof BytesRef ? ((BytesRef) value).utf8ToString() : value.toString(); - return Regex.simpleMatch(pattern, indexName); - } - @Override public IndexFieldData.Builder fielddataBuilder(String fullyQualifiedIndexName) { return new ConstantIndexFieldData.Builder(mapperService -> fullyQualifiedIndexName); diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java index ec4d5ade9d568..1decda0d6338b 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java @@ -65,6 +65,7 @@ import java.util.function.BiConsumer; import java.util.function.BiFunction; import java.util.function.LongSupplier; +import java.util.function.Predicate; /** * Context object used to create lucene queries on the shard level. @@ -86,7 +87,9 @@ public class QueryShardContext extends QueryRewriteContext { private final IndexSearcher searcher; private boolean cacheable = true; private final SetOnce frozen = new SetOnce<>(); + private final Index fullyQualifiedIndex; + private final Predicate indexNameMatcher; private final Map namedQueries = new HashMap<>(); private boolean allowUnmappedFields; @@ -94,45 +97,48 @@ public class QueryShardContext extends QueryRewriteContext { private NestedScope nestedScope; public QueryShardContext(int shardId, - IndexSettings indexSettings, - BigArrays bigArrays, - BitsetFilterCache bitsetFilterCache, - BiFunction> indexFieldDataLookup, - MapperService mapperService, - SimilarityService similarityService, - ScriptService scriptService, - NamedXContentRegistry xContentRegistry, - NamedWriteableRegistry namedWriteableRegistry, - Client client, - IndexSearcher searcher, - LongSupplier nowInMillis, - String clusterAlias) { + IndexSettings indexSettings, + BigArrays bigArrays, + BitsetFilterCache bitsetFilterCache, + BiFunction> indexFieldDataLookup, + MapperService mapperService, + SimilarityService similarityService, + ScriptService scriptService, + NamedXContentRegistry xContentRegistry, + NamedWriteableRegistry namedWriteableRegistry, + Client client, + IndexSearcher searcher, + LongSupplier nowInMillis, + String clusterAlias, + Predicate indexNameMatcher) { this(shardId, indexSettings, bigArrays, bitsetFilterCache, indexFieldDataLookup, mapperService, similarityService, - scriptService, xContentRegistry, namedWriteableRegistry, client, searcher, nowInMillis, + scriptService, xContentRegistry, namedWriteableRegistry, client, searcher, nowInMillis, indexNameMatcher, new Index(RemoteClusterAware.buildRemoteIndexName(clusterAlias, indexSettings.getIndex().getName()), indexSettings.getIndex().getUUID())); } public QueryShardContext(QueryShardContext source) { this(source.shardId, source.indexSettings, source.bigArrays, source.bitsetFilterCache, source.indexFieldDataService, - source.mapperService, source.similarityService, source.scriptService, source.getXContentRegistry(), - source.getWriteableRegistry(), source.client, source.searcher, source.nowInMillis, source.fullyQualifiedIndex); + source.mapperService, source.similarityService, source.scriptService, source.getXContentRegistry(), + source.getWriteableRegistry(), source.client, source.searcher, source.nowInMillis, source.indexNameMatcher, + source.fullyQualifiedIndex); } private QueryShardContext(int shardId, - IndexSettings indexSettings, - BigArrays bigArrays, - BitsetFilterCache bitsetFilterCache, - BiFunction> indexFieldDataLookup, - MapperService mapperService, - SimilarityService similarityService, - ScriptService scriptService, - NamedXContentRegistry xContentRegistry, - NamedWriteableRegistry namedWriteableRegistry, - Client client, - IndexSearcher searcher, - LongSupplier nowInMillis, - Index fullyQualifiedIndex) { + IndexSettings indexSettings, + BigArrays bigArrays, + BitsetFilterCache bitsetFilterCache, + BiFunction> indexFieldDataLookup, + MapperService mapperService, + SimilarityService similarityService, + ScriptService scriptService, + NamedXContentRegistry xContentRegistry, + NamedWriteableRegistry namedWriteableRegistry, + Client client, + IndexSearcher searcher, + LongSupplier nowInMillis, + Predicate indexNameMatcher, + Index fullyQualifiedIndex) { super(xContentRegistry, namedWriteableRegistry, client, nowInMillis); this.shardId = shardId; this.similarityService = similarityService; @@ -145,6 +151,7 @@ private QueryShardContext(int shardId, this.scriptService = scriptService; this.indexSettings = indexSettings; this.searcher = searcher; + this.indexNameMatcher = indexNameMatcher; this.fullyQualifiedIndex = fullyQualifiedIndex; } @@ -284,6 +291,14 @@ public Version indexVersionCreated() { return indexSettings.getIndexVersionCreated(); } + /** + * Given an index pattern, checks whether it matches against the current shard. The pattern + * may represent a fully qualified index name if the search targets remote shards. + */ + public boolean indexMatches(String pattern) { + return indexNameMatcher.test(pattern); + } + public ParsedQuery toQuery(QueryBuilder queryBuilder) { return toQuery(queryBuilder, q -> { Query query = q.toQuery(this); diff --git a/server/src/main/java/org/elasticsearch/index/query/SearchIndexNameMatcher.java b/server/src/main/java/org/elasticsearch/index/query/SearchIndexNameMatcher.java new file mode 100644 index 0000000000000..b2329d1d54c83 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/index/query/SearchIndexNameMatcher.java @@ -0,0 +1,84 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.query; + +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.transport.RemoteClusterAware; + +import java.util.function.Predicate; + +/** + * A predicate that checks whether an index pattern matches the current search shard target. + */ +public class SearchIndexNameMatcher implements Predicate { + private final String indexName; + private final String clusterAlias; + private final ClusterService clusterService; + private final IndexNameExpressionResolver expressionResolver; + + /** + * Creates a new index name matcher. + * + * @param indexName he name of the local index. + * @param clusterAlias the cluster alias of this search shard target. If it is a local target, the alias + * should be null or equal to {@link RemoteClusterAware#LOCAL_CLUSTER_GROUP_KEY}. + * @param clusterService the cluster service. + */ + public SearchIndexNameMatcher(String indexName, + String clusterAlias, + ClusterService clusterService) { + this.indexName = indexName; + this.clusterAlias = RemoteClusterAware.LOCAL_CLUSTER_GROUP_KEY.equals(clusterAlias) ? null : clusterAlias; + this.clusterService = clusterService; + this.expressionResolver = new IndexNameExpressionResolver(); + } + + /** + * Given an index pattern, checks whether it matches against the current shard. + * + * If this shard represents a remote shard target, then in order to match the pattern contain + * the separator ':', and must match on both the cluster alias and index name. + */ + public boolean test(String pattern) { + int separatorIndex = pattern.indexOf(RemoteClusterAware.REMOTE_CLUSTER_INDEX_SEPARATOR); + if (separatorIndex < 0) { + return clusterAlias == null && matchesIndex(pattern); + } else { + String clusterPattern = pattern.substring(0, separatorIndex); + String indexPattern = pattern.substring(separatorIndex + 1); + + return Regex.simpleMatch(clusterPattern, clusterAlias) && matchesIndex(indexPattern); + } + } + + private boolean matchesIndex(String pattern) { + String[] concreteIndices = expressionResolver.concreteIndexNames( + clusterService.state(), IndicesOptions.lenientExpandOpen(), pattern); + for (String index : concreteIndices) { + if (Regex.simpleMatch(index, indexName)) { + return true; + } + } + return false; + } +} diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index bcc4364057361..2000467f5c1e2 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -43,6 +43,7 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.CheckedFunction; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -185,6 +186,7 @@ public class IndicesService extends AbstractLifecycleComponent private final CircuitBreakerService circuitBreakerService; private final BigArrays bigArrays; private final ScriptService scriptService; + private final ClusterService clusterService; private final Client client; private volatile Map indices = emptyMap(); private final Map> pendingDeletes = new HashMap<>(); @@ -212,7 +214,7 @@ public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvi AnalysisRegistry analysisRegistry, IndexNameExpressionResolver indexNameExpressionResolver, MapperRegistry mapperRegistry, NamedWriteableRegistry namedWriteableRegistry, ThreadPool threadPool, IndexScopedSettings indexScopedSettings, CircuitBreakerService circuitBreakerService, BigArrays bigArrays, - ScriptService scriptService, Client client, MetaStateService metaStateService, + ScriptService scriptService, ClusterService clusterService, Client client, MetaStateService metaStateService, Collection>> engineFactoryProviders, Map directoryFactories) { this.settings = settings; @@ -234,6 +236,7 @@ public IndicesService(Settings settings, PluginsService pluginsService, NodeEnvi this.circuitBreakerService = circuitBreakerService; this.bigArrays = bigArrays; this.scriptService = scriptService; + this.clusterService = clusterService; this.client = client; this.indicesFieldDataCache = new IndicesFieldDataCache(settings, new IndexFieldDataCache.Listener() { @Override @@ -550,6 +553,7 @@ private synchronized IndexService createIndexService(IndexService.IndexCreationC bigArrays, threadPool, scriptService, + clusterService, client, indicesQueryCache, mapperRegistry, diff --git a/server/src/main/java/org/elasticsearch/node/Node.java b/server/src/main/java/org/elasticsearch/node/Node.java index 50cb468b48744..881c88b8b7775 100644 --- a/server/src/main/java/org/elasticsearch/node/Node.java +++ b/server/src/main/java/org/elasticsearch/node/Node.java @@ -424,10 +424,10 @@ protected Node( .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); final IndicesService indicesService = - new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry, analysisModule.getAnalysisRegistry(), - clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry, - threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, - scriptModule.getScriptService(), client, metaStateService, engineFactoryProviders, indexStoreFactories); + new IndicesService(settings, pluginsService, nodeEnvironment, xContentRegistry, analysisModule.getAnalysisRegistry(), + clusterModule.getIndexNameExpressionResolver(), indicesModule.getMapperRegistry(), namedWriteableRegistry, + threadPool, settingsModule.getIndexScopedSettings(), circuitBreakerService, bigArrays, scriptModule.getScriptService(), + clusterService, client, metaStateService, engineFactoryProviders, indexStoreFactories); final AliasValidator aliasValidator = new AliasValidator(); diff --git a/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java index d052fa365beb2..7c8d7b902fbfb 100644 --- a/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -150,7 +150,7 @@ public void tearDown() throws Exception { private IndexService newIndexService(IndexModule module) throws IOException { return module.newIndexService(CREATE_INDEX, nodeEnvironment, xContentRegistry(), deleter, circuitBreakerService, bigArrays, - threadPool, scriptService, null, indicesQueryCache, mapperRegistry, + threadPool, scriptService, clusterService, null, indicesQueryCache, mapperRegistry, new IndicesFieldDataCache(settings, listener), writableRegistry()); } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java index 479f4d7fc5558..6ac59169ad908 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/DateFieldTypeTests.java @@ -179,7 +179,7 @@ public void testTermQuery() { QueryShardContext context = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, - xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null); + xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null, null); MappedFieldType ft = createDefaultFieldType(); ft.setName("field"); String date = "2015-10-12T14:10:55"; @@ -202,7 +202,7 @@ public void testRangeQuery() throws IOException { QueryShardContext context = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(), - null, null, () -> nowInMillis, null); + null, null, () -> nowInMillis, null, null); MappedFieldType ft = createDefaultFieldType(); ft.setName("field"); String date1 = "2015-10-12T14:10:55"; diff --git a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java index 9bbeecdfc8f51..1a9460115f056 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/FieldNamesFieldTypeTests.java @@ -68,7 +68,7 @@ public void testTermQuery() { QueryShardContext queryShardContext = new QueryShardContext(0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, - null, null, null, null, null, null, () -> 0L, null); + null, null, null, null, null, null, () -> 0L, null, null); fieldNamesFieldType.setEnabled(true); Query termQuery = fieldNamesFieldType.termQuery("field_name", queryShardContext); assertEquals(new TermQuery(new Term(FieldNamesFieldMapper.CONTENT_TYPE, "field_name")), termQuery); diff --git a/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java index 82f0edf24f4da..11b365ff16e2b 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/IndexFieldTypeTests.java @@ -21,11 +21,14 @@ import org.apache.lucene.index.IndexOptions; import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; -import org.elasticsearch.index.Index; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.regex.Regex; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.query.QueryShardContext; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; +import java.util.function.Predicate; public class IndexFieldTypeTests extends FieldTypeTestCase { @@ -62,12 +65,15 @@ public void testWildcardQuery() { } private QueryShardContext createContext() { - QueryShardContext context = mock(QueryShardContext.class); + IndexMetaData indexMetaData = IndexMetaData.builder("index") + .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) + .numberOfShards(1) + .numberOfReplicas(0) + .build(); + IndexSettings indexSettings = new IndexSettings(indexMetaData, Settings.EMPTY); - Index index = new Index("index", "123"); - when(context.getFullyQualifiedIndex()).thenReturn(index); - when(context.index()).thenReturn(index); - - return context; + Predicate indexNameMatcher = pattern -> Regex.simpleMatch(pattern, "index"); + return new QueryShardContext(0, indexSettings, null, null, null, null, null, null, xContentRegistry(), writableRegistry(), + null, null, System::currentTimeMillis, null, indexNameMatcher); } } diff --git a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java index 16fe2ceee8f53..79ab18afbd590 100644 --- a/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java +++ b/server/src/test/java/org/elasticsearch/index/mapper/RangeFieldTypeTests.java @@ -229,7 +229,7 @@ private QueryShardContext createContext() { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT).build(); IndexSettings idxSettings = IndexSettingsModule.newIndexSettings(randomAlphaOfLengthBetween(1, 10), indexSettings); return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, - xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null); + xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null, null); } public void testDateRangeQueryUsingMappingFormat() { diff --git a/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java index 15f9b52d23bdb..4974a6cb69020 100644 --- a/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/IntervalQueryBuilderTests.java @@ -372,7 +372,7 @@ public FactoryType compile(Script script, ScriptContext mappedFieldType.fielddataBuilder(idxName).build(indexSettings, mappedFieldType, null, null, null), mapperService, null, null, NamedXContentRegistry.EMPTY, new NamedWriteableRegistry(Collections.emptyList()), - null, null, () -> nowInMillis, clusterAlias); + null, null, () -> nowInMillis, clusterAlias, null); } } diff --git a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java index f4d7c90488f15..83ab9c8e62bb4 100644 --- a/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/RangeQueryRewriteTests.java @@ -41,7 +41,7 @@ public void testRewriteMissingField() throws Exception { IndexReader reader = new MultiReader(); 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); + null, new IndexSearcher(reader), null, null, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); assertEquals(Relation.DISJOINT, range.getRelation(context)); } @@ -57,9 +57,8 @@ public void testRewriteMissingReader() throws Exception { .endObject().endObject()); indexService.mapperService().merge("type", new CompressedXContent(mapping), MergeReason.MAPPING_UPDATE); - QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE, - null, null, indexService.mapperService(), null, null, - xContentRegistry(), writableRegistry(), null, null, null, null); + QueryRewriteContext context = new QueryShardContext(0, indexService.getIndexSettings(), null, null, null, + indexService.mapperService(), null, null, xContentRegistry(), writableRegistry(), null, null, null, null, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // can't make assumptions on a missing reader, so it must return INTERSECT assertEquals(Relation.INTERSECTS, range.getRelation(context)); @@ -79,7 +78,7 @@ public void testRewriteEmptyReader() throws Exception { IndexReader reader = new MultiReader(); 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); + null, new IndexSearcher(reader), null, null, null); RangeQueryBuilder range = new RangeQueryBuilder("foo"); // no values -> DISJOINT assertEquals(Relation.DISJOINT, range.getRelation(context)); diff --git a/server/src/test/java/org/elasticsearch/index/query/SearchIndexNameMatcherTests.java b/server/src/test/java/org/elasticsearch/index/query/SearchIndexNameMatcherTests.java new file mode 100644 index 0000000000000..a796586bcf564 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/query/SearchIndexNameMatcherTests.java @@ -0,0 +1,90 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.index.query; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.AliasMetaData; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESTestCase; +import org.junit.Before; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class SearchIndexNameMatcherTests extends ESTestCase { + private SearchIndexNameMatcher matcher; + private SearchIndexNameMatcher remoteMatcher; + + @Before + public void setUpMatchers() { + MetaData.Builder metaDataBuilder = MetaData.builder() + .put(indexBuilder("index1").putAlias(AliasMetaData.builder("alias"))) + .put(indexBuilder("index2").putAlias(AliasMetaData.builder("alias"))) + .put(indexBuilder("index3")); + ClusterState state = ClusterState.builder(new ClusterName("_name")).metaData(metaDataBuilder).build(); + + ClusterService clusterService = mock(ClusterService.class); + when(clusterService.state()).thenReturn(state); + + matcher = new SearchIndexNameMatcher("index1", "", clusterService); + remoteMatcher = new SearchIndexNameMatcher("index1", "cluster", clusterService); + } + + private static IndexMetaData.Builder indexBuilder(String index) { + Settings.Builder settings = settings(Version.CURRENT). + put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0); + return IndexMetaData.builder(index).settings(settings); + } + + public void testLocalIndex() { + assertTrue(matcher.test("index1")); + assertTrue(matcher.test("ind*x1")); + assertFalse(matcher.test("index2")); + + assertTrue(matcher.test("alias")); + assertTrue(matcher.test("*lias")); + + assertFalse(matcher.test("cluster:index1")); + } + + public void testRemoteIndex() { + assertTrue(remoteMatcher.test("cluster:index1")); + assertTrue(remoteMatcher.test("cluster:ind*x1")); + assertTrue(remoteMatcher.test("*luster:ind*x1")); + assertFalse(remoteMatcher.test("cluster:index2")); + + assertTrue(remoteMatcher.test("cluster:alias")); + assertTrue(remoteMatcher.test("cluster:*lias")); + + assertFalse(remoteMatcher.test("index1")); + assertFalse(remoteMatcher.test("alias")); + + assertFalse(remoteMatcher.test("*index1")); + assertFalse(remoteMatcher.test("*alias")); + assertFalse(remoteMatcher.test("cluster*")); + assertFalse(remoteMatcher.test("cluster*index1")); + } +} diff --git a/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java index fbd707b4c2ae1..f0869c4bd86b0 100644 --- a/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/SimpleQueryStringBuilderTests.java @@ -27,7 +27,6 @@ import org.apache.lucene.search.BoostQuery; import org.apache.lucene.search.DisjunctionMaxQuery; import org.apache.lucene.search.FuzzyQuery; -import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.PrefixQuery; @@ -413,15 +412,6 @@ public void testMinimumShouldMatch() throws IOException { } } - public void testIndexMetaField() throws IOException { - QueryShardContext shardContext = createShardContext(); - SimpleQueryStringBuilder simpleQueryStringBuilder = new SimpleQueryStringBuilder(getIndex().getName()); - simpleQueryStringBuilder.field("_index"); - Query query = simpleQueryStringBuilder.toQuery(shardContext); - assertThat(query, notNullValue()); - assertThat(query, instanceOf(MatchAllDocsQuery.class)); - } - public void testExpandedTerms() throws Exception { // Prefix Query query = new SimpleQueryStringBuilder("aBc*") diff --git a/server/src/test/java/org/elasticsearch/index/query/WildcardQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/WildcardQueryBuilderTests.java index caab692b9cf6b..7cd1ca50de3a1 100644 --- a/server/src/test/java/org/elasticsearch/index/query/WildcardQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/WildcardQueryBuilderTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.index.query; -import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.apache.lucene.search.WildcardQuery; @@ -135,20 +134,6 @@ public void testParseFailsWithMultipleFields() throws IOException { assertEquals("[wildcard] query doesn't support multiple fields, found [user1] and [user2]", e.getMessage()); } - public void testIndexWildcard() throws IOException { - QueryShardContext context = createShardContext(); - String index = context.getFullyQualifiedIndex().getName(); - - Query query = new WildcardQueryBuilder("_index", index).doToQuery(context); - assertThat(query instanceof MatchAllDocsQuery, equalTo(true)); - - query = new WildcardQueryBuilder("_index", index + "*").doToQuery(context); - assertThat(query instanceof MatchAllDocsQuery, equalTo(true)); - - query = new WildcardQueryBuilder("_index", "index_" + index + "*").doToQuery(context); - assertThat(query instanceof MatchNoDocsQuery, equalTo(true)); - } - public void testTypeField() throws IOException { WildcardQueryBuilder builder = QueryBuilders.wildcardQuery("_type", "doc*"); builder.doToQuery(createShardContext()); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java index 155b7cb8f9772..6fc5561f6d6f7 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/histogram/ExtendedBoundsTests.java @@ -97,7 +97,7 @@ public void testParseAndValidate() { QueryShardContext qsc = new QueryShardContext(0, new IndexSettings(IndexMetaData.builder("foo").settings(indexSettings).build(), indexSettings), BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), writableRegistry(), - null, null, () -> now, null); + null, null, () -> now, null, null); DateFormatter formatter = DateFormatter.forPattern("dateOptionalTime"); DocValueFormat format = new DocValueFormat.DateTime(formatter, ZoneOffset.UTC, DateFieldMapper.Resolution.MILLISECONDS); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java index 7203b5dd44357..9d0d1d69f023c 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/ScriptedMetricAggregatorTests.java @@ -426,6 +426,6 @@ protected QueryShardContext queryShardContextMock(IndexSearcher searcher, Map engines = Collections.singletonMap(scriptEngine.getType(), scriptEngine); ScriptService scriptService = new ScriptService(Settings.EMPTY, engines, ScriptModule.CORE_CONTEXTS); return new QueryShardContext(0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, null, scriptService, - xContentRegistry(), writableRegistry(), null, null, System::currentTimeMillis, null); + xContentRegistry(), writableRegistry(), null, null, System::currentTimeMillis, null, null); } } diff --git a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java index e02ee4c22ba9e..36c0d7ea12375 100644 --- a/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/fetch/subphase/highlight/HighlightBuilderTests.java @@ -280,7 +280,7 @@ public void testBuildSearchContextHighlight() throws IOException { // shard context will only need indicesQueriesRegistry for building Query objects nested in highlighter QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, xContentRegistry(), namedWriteableRegistry, - null, null, System::currentTimeMillis, null) { + null, null, System::currentTimeMillis, null, null) { @Override public MappedFieldType fieldMapper(String name) { TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name); diff --git a/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java b/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java index a68ee8c27662b..302bffd668997 100644 --- a/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/search/rescore/QueryRescorerBuilderTests.java @@ -144,7 +144,7 @@ public void testBuildRescoreSearchContext() throws ElasticsearchParseException, // shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, - xContentRegistry(), namedWriteableRegistry, null, null, () -> nowInMillis, null) { + xContentRegistry(), namedWriteableRegistry, null, null, () -> nowInMillis, null, null) { @Override public MappedFieldType fieldMapper(String name) { TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name); @@ -188,7 +188,7 @@ public void testRewritingKeepsSettings() throws IOException { // shard context will only need indicesQueriesRegistry for building Query objects nested in query rescorer QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, - xContentRegistry(), namedWriteableRegistry, null, null, () -> nowInMillis, null) { + xContentRegistry(), namedWriteableRegistry, null, null, () -> nowInMillis, null, null) { @Override public MappedFieldType fieldMapper(String name) { TextFieldMapper.Builder builder = new TextFieldMapper.Builder(name); diff --git a/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java b/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java index caa706e0aaf57..7256a46715a05 100644 --- a/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/sort/AbstractSortTestCase.java @@ -192,7 +192,7 @@ protected QueryShardContext createMockShardContext() { return builder.build(idxSettings, fieldType, new IndexFieldDataCache.None(), null, null); }; return new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, bitsetFilterCache, indexFieldDataLookup, - null, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, () -> randomNonNegativeLong(), null) { + null, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, () -> randomNonNegativeLong(), null, null) { @Override public MappedFieldType fieldMapper(String name) { diff --git a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java index a8298b76295ea..48936b4d201e3 100644 --- a/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java +++ b/server/src/test/java/org/elasticsearch/search/suggest/AbstractSuggestionBuilderTestCase.java @@ -181,7 +181,7 @@ public void testBuild() throws IOException { ((Script) invocation.getArguments()[0]).getIdOrCode())); QueryShardContext mockShardContext = new QueryShardContext(0, idxSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, null, scriptService, xContentRegistry(), namedWriteableRegistry, null, null, - System::currentTimeMillis, null); + System::currentTimeMillis, null, null); SuggestionContext suggestionContext = suggestionBuilder.build(mockShardContext); assertEquals(toBytesRef(suggestionBuilder.text()), suggestionContext.getText()); diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java index e7e65f54bb711..f88a5b7a484dc 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotResiliencyTests.java @@ -976,6 +976,7 @@ public void onFailure(final Exception e) { new NoneCircuitBreakerService(), bigArrays, scriptService, + clusterService, client, new MetaStateService(nodeEnv, namedXContentRegistry), Collections.emptyList(), diff --git a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java index 087b452232561..199f9b055393c 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/search/aggregations/AggregatorTestCase.java @@ -274,14 +274,14 @@ protected MapperService mapperServiceMock() { * sub-tests that need a more complex mock can overwrite this */ protected QueryShardContext queryShardContextMock(IndexSearcher searcher, - MapperService mapperService, - IndexSettings indexSettings, - CircuitBreakerService circuitBreakerService) { + MapperService mapperService, + IndexSettings indexSettings, + CircuitBreakerService circuitBreakerService) { return new QueryShardContext(0, indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, getIndexFieldDataLookup(mapperService, circuitBreakerService), mapperService, null, getMockScriptService(), xContentRegistry(), - writableRegistry(), null, searcher, System::currentTimeMillis, null); + writableRegistry(), null, searcher, System::currentTimeMillis, null, null); } /** diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java index 39c6a777a21e7..69cfc1f6dc0cd 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java @@ -424,7 +424,7 @@ public void close() throws IOException { 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); + namedWriteableRegistry, this.client, searcher, () -> nowInMillis, null, null); } ScriptModule createScriptModule(List scriptPlugins) { diff --git a/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java b/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java index 668495f6f70a5..8a8842487f14a 100644 --- a/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java +++ b/test/framework/src/test/java/org/elasticsearch/search/MockSearchServiceTests.java @@ -43,7 +43,7 @@ public void testAssertNoInFlightContext() { final long nowInMillis = randomNonNegativeLong(); SearchContext s = new TestSearchContext(new QueryShardContext(0, new IndexSettings(EMPTY_INDEX_METADATA, Settings.EMPTY), BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, - xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null)) { + xContentRegistry(), writableRegistry(), null, null, () -> nowInMillis, null, null)) { @Override public SearchShardTarget shardTarget() { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java index 1a06f943083c7..a8bf5272e94fb 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/DocumentSubsetBitsetCacheTests.java @@ -240,7 +240,7 @@ private void runTestOnIndex(CheckedBiConsumer nowInMillis, null); + client, new IndexSearcher(directoryReader), () -> nowInMillis, null, null); body.accept(context, leaf); } } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java index 8214d327491fc..ca49e4ae4a3ed 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/security/authz/accesscontrol/SecurityIndexReaderWrapperIntegrationTests.java @@ -85,7 +85,7 @@ public void testDLS() throws Exception { final long nowInMillis = randomNonNegativeLong(); QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, null, null, xContentRegistry(), writableRegistry(), - client, null, () -> nowInMillis, null); + client, null, () -> nowInMillis, null, null); QueryShardContext queryShardContext = spy(realQueryShardContext); DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); XPackLicenseState licenseState = mock(XPackLicenseState.class); @@ -200,7 +200,7 @@ public void testDLSWithLimitedPermissions() throws Exception { final long nowInMillis = randomNonNegativeLong(); QueryShardContext realQueryShardContext = new QueryShardContext(shardId.id(), indexSettings, BigArrays.NON_RECYCLING_INSTANCE, null, null, mapperService, null, null, xContentRegistry(), writableRegistry(), - client, null, () -> nowInMillis, null); + client, null, () -> nowInMillis, null, null); QueryShardContext queryShardContext = spy(realQueryShardContext); DocumentSubsetBitsetCache bitsetCache = new DocumentSubsetBitsetCache(Settings.EMPTY); diff --git a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java index fbd8cb5f90700..1b3bbd9bcec24 100644 --- a/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java +++ b/x-pack/plugin/rollup/src/test/java/org/elasticsearch/xpack/rollup/job/RollupIndexerIndexingTests.java @@ -91,7 +91,7 @@ private void setup() { settings = createIndexSettings(); queryShardContext = new QueryShardContext(0, settings, BigArrays.NON_RECYCLING_INSTANCE, null, null, null, null, null, - null, null, null, null, () -> 0L, null); + null, null, null, null, () -> 0L, null, null); } public void testSimpleDateHisto() throws Exception {