diff --git a/docs/reference/migration/migrate_7_5.asciidoc b/docs/reference/migration/migrate_7_5.asciidoc
new file mode 100644
index 0000000000000..2334ce8aa5a8a
--- /dev/null
+++ b/docs/reference/migration/migrate_7_5.asciidoc
@@ -0,0 +1,30 @@
+[[breaking-changes-7.5]]
+== Breaking changes in 7.5
+++++
+7.5
+++++
+
+This section discusses the changes that you need to be aware of when migrating
+your application to Elasticsearch 7.5.
+
+See also <> and <>.
+
+coming[7.5.0]
+
+//NOTE: The notable-breaking-changes tagged regions are re-used in the
+//Installation and Upgrade Guide
+
+//tag::notable-breaking-changes[]
+
+//end::notable-breaking-changes[]
+
+[discrete]
+[[breaking_75_search_changes]]
+=== Search Changes
+
+[discrete]
+==== Stricter checking for wildcard queries on _index
+Previously, a wildcard query on the `_index` field matched directly against the
+fully-qualified index name. Now, in order to match against remote indices like
+i`cluster:index`, the query must contain a colon, as in `cl*ster:inde*`. This
+behavior aligns with the way indices are matched in the search endpoint.
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 da470a04afabd..5e2ac0dbac605 100644
--- a/server/src/main/java/org/elasticsearch/index/IndexService.java
+++ b/server/src/main/java/org/elasticsearch/index/IndexService.java
@@ -32,6 +32,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;
@@ -134,6 +136,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;
@@ -151,6 +154,7 @@ public IndexService(
BigArrays bigArrays,
ThreadPool threadPool,
ScriptService scriptService,
+ ClusterService clusterService,
Client client,
QueryCache queryCache,
IndexStorePlugin.DirectoryFactory directoryFactory,
@@ -201,6 +205,7 @@ public IndexService(
this.bigArrays = bigArrays;
this.threadPool = threadPool;
this.scriptService = scriptService;
+ this.clusterService = clusterService;
this.client = client;
this.eventListener = eventListener;
this.nodeEnv = nodeEnv;
@@ -530,9 +535,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 a631ea319b40a..b6eea750748d2 100644
--- a/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java
+++ b/server/src/main/java/org/elasticsearch/index/query/QueryShardContext.java
@@ -69,6 +69,7 @@
import java.util.function.BiConsumer;
import java.util.function.BiFunction;
import java.util.function.LongSupplier;
+import java.util.function.Predicate;
import static java.util.Collections.unmodifiableMap;
@@ -93,7 +94,9 @@ public class QueryShardContext extends QueryRewriteContext {
private String[] types = Strings.EMPTY_ARRAY;
private boolean cacheable = true;
private final SetOnce frozen = new SetOnce<>();
+
private final Index fullyQualifiedIndex;
+ private final Predicate indexNameMatcher;
public void setTypes(String... types) {
this.types = types;
@@ -109,45 +112,48 @@ public String[] getTypes() {
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;
@@ -160,6 +166,7 @@ private QueryShardContext(int shardId,
this.scriptService = scriptService;
this.indexSettings = indexSettings;
this.searcher = searcher;
+ this.indexNameMatcher = indexNameMatcher;
this.fullyQualifiedIndex = fullyQualifiedIndex;
}
@@ -311,6 +318,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 38ab71495210f..b6c87e576bd2c 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;
@@ -186,6 +187,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<>();
@@ -213,7 +215,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;
@@ -235,6 +237,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
@@ -556,6 +559,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 86a55ceb47a3f..efa7ddcd657a2 100644
--- a/server/src/main/java/org/elasticsearch/node/Node.java
+++ b/server/src/main/java/org/elasticsearch/node/Node.java
@@ -427,10 +427,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 fe8956c8b683f..e13b63516e87c 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;
@@ -417,15 +416,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 65d2e92555cfb..a1f669558a584 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 995cfa3b1c993..accf23a9644aa 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 a09cb4b0dfa63..28ca23df12401 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 d0289c7fa9774..f60c3f0774089 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 5a802414864a1..10c2188d47874 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 186fc94ee57c6..c90d65ba0a0fa 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
@@ -272,14 +272,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 98c66fc3d30d8..059796640a218 100644
--- a/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java
+++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractBuilderTestCase.java
@@ -425,7 +425,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 f78b9c2aa6f82..a50c39d4e6aac 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
@@ -241,7 +241,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 7f5a8232a6d8b..492d24b88f06e 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
@@ -93,7 +93,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 {