From fd937ebf77c209bf32d9f17b5772bc2ac163994e Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 12 Dec 2024 16:18:41 +0100 Subject: [PATCH 1/6] [8.x] Handle all exceptions in data nodes can match (#117469) (#118533) (#118570) * Handle all exceptions in data nodes can match (#117469) During the can match phase, prior to the query phase, we may have exceptions that are returned back to the coordinating node, handled gracefully as if the shard returned canMatch=true. During the query phase, we perform an additional rewrite and can match phase to eventually shortcut the query phase for the shard. That needs to handle exceptions as well. Currently, an exception there causes shard failures, while we should rather go ahead and execute the query on the shard. Instead of adding another try catch on consumers code, this commit adds exception handling to the method itself so that it can no longer throw exceptions and similar mistakes can no longer be made in the future. At the same time, this commit makes the can match method more easily testable without requiring a full-blown SearchService instance. Closes #104994 * fix compile --- docs/changelog/117469.yaml | 6 + .../elasticsearch/search/SearchService.java | 182 +- .../search/SearchServiceSingleNodeTests.java | 3011 +++++++++++++++ .../search/SearchServiceTests.java | 3257 ++--------------- .../action/TransportTermsEnumAction.java | 2 +- .../index/engine/frozen/FrozenIndexTests.java | 41 +- .../BaseSearchableSnapshotsIntegTestCase.java | 12 + ...pshotsCanMatchOnCoordinatorIntegTests.java | 14 - .../SearchableSnapshotsSearchIntegTests.java | 129 + 9 files changed, 3521 insertions(+), 3133 deletions(-) create mode 100644 docs/changelog/117469.yaml create mode 100644 server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java create mode 100644 x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSearchIntegTests.java diff --git a/docs/changelog/117469.yaml b/docs/changelog/117469.yaml new file mode 100644 index 0000000000000..cfb14f78cb578 --- /dev/null +++ b/docs/changelog/117469.yaml @@ -0,0 +1,6 @@ +pr: 117469 +summary: Handle exceptions in query phase can match +area: Search +type: bug +issues: + - 104994 diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 469d798e55472..f86aada356537 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -149,6 +149,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; +import java.util.function.Function; import java.util.function.LongSupplier; import java.util.function.Supplier; @@ -555,16 +557,17 @@ public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, // check if we can shortcut the query phase entirely. if (orig.canReturnNullResponseIfMatchNoDocs()) { assert orig.scroll() == null; - final CanMatchShardResponse canMatchResp; - try { - ShardSearchRequest clone = new ShardSearchRequest(orig); - canMatchResp = canMatch(clone, false); - } catch (Exception exc) { - l.onFailure(exc); - return; - } + ShardSearchRequest clone = new ShardSearchRequest(orig); + CanMatchContext canMatchContext = new CanMatchContext( + clone, + indicesService::indexServiceSafe, + this::findReaderContext, + defaultKeepAlive, + maxKeepAlive + ); + CanMatchShardResponse canMatchResp = canMatch(canMatchContext, false); if (canMatchResp.canMatch() == false) { - l.onResponse(QuerySearchResult.nullInstance()); + listener.onResponse(QuerySearchResult.nullInstance()); return; } } @@ -1212,10 +1215,14 @@ public void freeAllScrollContexts() { } private long getKeepAlive(ShardSearchRequest request) { + return getKeepAlive(request, defaultKeepAlive, maxKeepAlive); + } + + private static long getKeepAlive(ShardSearchRequest request, long defaultKeepAlive, long maxKeepAlive) { if (request.scroll() != null) { - return getScrollKeepAlive(request.scroll()); + return getScrollKeepAlive(request.scroll(), defaultKeepAlive, maxKeepAlive); } else if (request.keepAlive() != null) { - checkKeepAliveLimit(request.keepAlive().millis()); + checkKeepAliveLimit(request.keepAlive().millis(), maxKeepAlive); return request.keepAlive().getMillis(); } else { return request.readerId() == null ? defaultKeepAlive : -1; @@ -1223,14 +1230,22 @@ private long getKeepAlive(ShardSearchRequest request) { } private long getScrollKeepAlive(Scroll scroll) { + return getScrollKeepAlive(scroll, defaultKeepAlive, maxKeepAlive); + } + + private static long getScrollKeepAlive(Scroll scroll, long defaultKeepAlive, long maxKeepAlive) { if (scroll != null && scroll.keepAlive() != null) { - checkKeepAliveLimit(scroll.keepAlive().millis()); + checkKeepAliveLimit(scroll.keepAlive().millis(), maxKeepAlive); return scroll.keepAlive().getMillis(); } return defaultKeepAlive; } private void checkKeepAliveLimit(long keepAlive) { + checkKeepAliveLimit(keepAlive, maxKeepAlive); + } + + private static void checkKeepAliveLimit(long keepAlive, long maxKeepAlive) { if (keepAlive > maxKeepAlive) { throw new IllegalArgumentException( "Keep alive for request (" @@ -1689,6 +1704,7 @@ public void canMatch(CanMatchNodeRequest request, ActionListener responses = new ArrayList<>(shardLevelRequests.size()); for (var shardLevelRequest : shardLevelRequests) { try { + // TODO remove the exception handling as it's now in canMatch itself responses.add(new CanMatchNodeResponse.ResponseOrFailure(canMatch(request.createShardSearchRequest(shardLevelRequest)))); } catch (Exception e) { responses.add(new CanMatchNodeResponse.ResponseOrFailure(e)); @@ -1700,82 +1716,145 @@ public void canMatch(CanMatchNodeRequest request, ActionListener indexServiceLookup; + private final BiFunction findReaderContext; + private final long defaultKeepAlive; + private final long maxKeepAlive; + + private IndexService indexService; + + CanMatchContext( + ShardSearchRequest request, + Function indexServiceLookup, + BiFunction findReaderContext, + long defaultKeepAlive, + long maxKeepAlive + ) { + this.request = request; + this.indexServiceLookup = indexServiceLookup; + this.findReaderContext = findReaderContext; + this.defaultKeepAlive = defaultKeepAlive; + this.maxKeepAlive = maxKeepAlive; + } + + long getKeepAlive() { + return SearchService.getKeepAlive(request, defaultKeepAlive, maxKeepAlive); + } + + ReaderContext findReaderContext() { + return findReaderContext.apply(request.readerId(), request); + } + + QueryRewriteContext getQueryRewriteContext(IndexService indexService) { + return indexService.newQueryRewriteContext(request::nowInMillis, request.getRuntimeMappings(), request.getClusterAlias()); + } + + SearchExecutionContext getSearchExecutionContext(Engine.Searcher searcher) { + return getIndexService().newSearchExecutionContext( + request.shardId().id(), + 0, + searcher, + request::nowInMillis, + request.getClusterAlias(), + request.getRuntimeMappings() + ); + } + + IndexShard getShard() { + return getIndexService().getShard(request.shardId().getId()); + } + + IndexService getIndexService() { + if (this.indexService == null) { + this.indexService = indexServiceLookup.apply(request.shardId().getIndex()); + } + return this.indexService; + } + } + + static CanMatchShardResponse canMatch(CanMatchContext canMatchContext, boolean checkRefreshPending) { + assert canMatchContext.request.searchType() == SearchType.QUERY_THEN_FETCH + : "unexpected search type: " + canMatchContext.request.searchType(); Releasable releasable = null; try { IndexService indexService; final boolean hasRefreshPending; final Engine.Searcher canMatchSearcher; - if (request.readerId() != null) { + if (canMatchContext.request.readerId() != null) { hasRefreshPending = false; ReaderContext readerContext; Engine.Searcher searcher; try { - readerContext = findReaderContext(request.readerId(), request); - releasable = readerContext.markAsUsed(getKeepAlive(request)); + readerContext = canMatchContext.findReaderContext(); + releasable = readerContext.markAsUsed(canMatchContext.getKeepAlive()); indexService = readerContext.indexService(); - if (canMatchAfterRewrite(request, indexService) == false) { + QueryRewriteContext queryRewriteContext = canMatchContext.getQueryRewriteContext(indexService); + if (queryStillMatchesAfterRewrite(canMatchContext.request, queryRewriteContext) == false) { return new CanMatchShardResponse(false, null); } searcher = readerContext.acquireSearcher(Engine.CAN_MATCH_SEARCH_SOURCE); } catch (SearchContextMissingException e) { - final String searcherId = request.readerId().getSearcherId(); + final String searcherId = canMatchContext.request.readerId().getSearcherId(); if (searcherId == null) { - throw e; + return new CanMatchShardResponse(true, null); } - indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); - if (canMatchAfterRewrite(request, indexService) == false) { + if (queryStillMatchesAfterRewrite( + canMatchContext.request, + canMatchContext.getQueryRewriteContext(canMatchContext.getIndexService()) + ) == false) { return new CanMatchShardResponse(false, null); } - IndexShard indexShard = indexService.getShard(request.shardId().getId()); - final Engine.SearcherSupplier searcherSupplier = indexShard.acquireSearcherSupplier(); + final Engine.SearcherSupplier searcherSupplier = canMatchContext.getShard().acquireSearcherSupplier(); if (searcherId.equals(searcherSupplier.getSearcherId()) == false) { searcherSupplier.close(); - throw e; + return new CanMatchShardResponse(true, null); } releasable = searcherSupplier; searcher = searcherSupplier.acquireSearcher(Engine.CAN_MATCH_SEARCH_SOURCE); } canMatchSearcher = searcher; } else { - indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); - if (canMatchAfterRewrite(request, indexService) == false) { + if (queryStillMatchesAfterRewrite( + canMatchContext.request, + canMatchContext.getQueryRewriteContext(canMatchContext.getIndexService()) + ) == false) { return new CanMatchShardResponse(false, null); } - IndexShard indexShard = indexService.getShard(request.shardId().getId()); - boolean needsWaitForRefresh = request.waitForCheckpoint() != UNASSIGNED_SEQ_NO; + boolean needsWaitForRefresh = canMatchContext.request.waitForCheckpoint() != UNASSIGNED_SEQ_NO; // If this request wait_for_refresh behavior, it is safest to assume a refresh is pending. Theoretically, // this can be improved in the future by manually checking that the requested checkpoint has already been refresh. // However, this will request modifying the engine to surface that information. + IndexShard indexShard = canMatchContext.getShard(); hasRefreshPending = needsWaitForRefresh || (indexShard.hasRefreshPending() && checkRefreshPending); canMatchSearcher = indexShard.acquireSearcher(Engine.CAN_MATCH_SEARCH_SOURCE); } try (canMatchSearcher) { - SearchExecutionContext context = indexService.newSearchExecutionContext( - request.shardId().id(), - 0, - canMatchSearcher, - request::nowInMillis, - request.getClusterAlias(), - request.getRuntimeMappings() - ); - final boolean canMatch = queryStillMatchesAfterRewrite(request, context); - final MinAndMax minMax; + SearchExecutionContext context = canMatchContext.getSearchExecutionContext(canMatchSearcher); + final boolean canMatch = queryStillMatchesAfterRewrite(canMatchContext.request, context); if (canMatch || hasRefreshPending) { - FieldSortBuilder sortBuilder = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); - minMax = sortBuilder != null ? FieldSortBuilder.getMinMaxOrNull(context, sortBuilder) : null; - } else { - minMax = null; + FieldSortBuilder sortBuilder = FieldSortBuilder.getPrimaryFieldSortOrNull(canMatchContext.request.source()); + final MinAndMax minMax = sortBuilder != null ? FieldSortBuilder.getMinMaxOrNull(context, sortBuilder) : null; + return new CanMatchShardResponse(true, minMax); } - return new CanMatchShardResponse(canMatch || hasRefreshPending, minMax); + return new CanMatchShardResponse(false, null); } + } catch (Exception e) { + return new CanMatchShardResponse(true, null); } finally { Releasables.close(releasable); } @@ -1788,15 +1867,6 @@ private CanMatchShardResponse canMatch(ShardSearchRequest request, boolean check * {@link MatchNoneQueryBuilder}. This allows us to avoid extra work for example making the shard search active and waiting for * refreshes. */ - private static boolean canMatchAfterRewrite(final ShardSearchRequest request, final IndexService indexService) throws IOException { - final QueryRewriteContext queryRewriteContext = indexService.newQueryRewriteContext( - request::nowInMillis, - request.getRuntimeMappings(), - request.getClusterAlias() - ); - return queryStillMatchesAfterRewrite(request, queryRewriteContext); - } - @SuppressWarnings("unchecked") public static boolean queryStillMatchesAfterRewrite(ShardSearchRequest request, QueryRewriteContext context) throws IOException { Rewriteable.rewrite(request.getRewriteable(), context, false); diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java new file mode 100644 index 0000000000000..1f2d129f0293c --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceSingleNodeTests.java @@ -0,0 +1,3011 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the "Elastic License + * 2.0", the "GNU Affero General Public License v3.0 only", and the "Server Side + * Public License v 1"; you may not use this file except in compliance with, at + * your election, the "Elastic License 2.0", the "GNU Affero General Public + * License v3.0 only", or the "Server Side Public License, v 1". + */ +package org.elasticsearch.search; + +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.FilterDirectoryReader; +import org.apache.lucene.index.LeafReader; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.search.ScoreDoc; +import org.apache.lucene.search.TopDocs; +import org.apache.lucene.search.TotalHitCountCollectorManager; +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteResponse; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.elasticsearch.action.search.ClearScrollRequest; +import org.elasticsearch.action.search.ClosePointInTimeRequest; +import org.elasticsearch.action.search.OpenPointInTimeRequest; +import org.elasticsearch.action.search.SearchPhaseController; +import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.search.SearchScrollRequest; +import org.elasticsearch.action.search.SearchShardTask; +import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.action.search.TransportClosePointInTimeAction; +import org.elasticsearch.action.search.TransportOpenPointInTimeAction; +import org.elasticsearch.action.search.TransportSearchAction; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.client.internal.Client; +import org.elasticsearch.cluster.metadata.IndexMetadata; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.TestShardRouting; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.breaker.CircuitBreakingException; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.core.AbstractRefCounted; +import org.elasticsearch.core.TimeValue; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexModule; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.MatchNoneQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryRewriteContext; +import org.elasticsearch.index.query.SearchExecutionContext; +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.index.search.stats.SearchStats; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.SearchOperationListener; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.indices.settings.InternalOrPrivateSettingsPlugin; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.script.MockScriptEngine; +import org.elasticsearch.script.MockScriptPlugin; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptType; +import org.elasticsearch.search.SearchService.ResultsType; +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.AggregationReduceContext; +import org.elasticsearch.search.aggregations.MultiBucketConsumerService; +import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.builder.PointInTimeBuilder; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.search.dfs.AggregatedDfs; +import org.elasticsearch.search.fetch.FetchSearchResult; +import org.elasticsearch.search.fetch.ShardFetchRequest; +import org.elasticsearch.search.fetch.ShardFetchSearchRequest; +import org.elasticsearch.search.fetch.subphase.FieldAndFormat; +import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.ContextIndexSearcher; +import org.elasticsearch.search.internal.ReaderContext; +import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.internal.ShardSearchContextId; +import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.search.query.NonCountingTermQuery; +import org.elasticsearch.search.query.QuerySearchRequest; +import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.search.query.SearchTimeoutException; +import org.elasticsearch.search.rank.RankBuilder; +import org.elasticsearch.search.rank.RankDoc; +import org.elasticsearch.search.rank.RankShardResult; +import org.elasticsearch.search.rank.TestRankBuilder; +import org.elasticsearch.search.rank.TestRankShardResult; +import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; +import org.elasticsearch.search.rank.context.RankFeaturePhaseRankCoordinatorContext; +import org.elasticsearch.search.rank.context.RankFeaturePhaseRankShardContext; +import org.elasticsearch.search.rank.feature.RankFeatureDoc; +import org.elasticsearch.search.rank.feature.RankFeatureResult; +import org.elasticsearch.search.rank.feature.RankFeatureShardRequest; +import org.elasticsearch.search.rank.feature.RankFeatureShardResult; +import org.elasticsearch.search.suggest.SuggestBuilder; +import org.elasticsearch.tasks.TaskCancelHelper; +import org.elasticsearch.tasks.TaskCancelledException; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xcontent.XContentBuilder; +import org.elasticsearch.xcontent.json.JsonXContent; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Semaphore; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.function.IntConsumer; +import java.util.function.Supplier; + +import static java.util.Collections.emptyList; +import static java.util.Collections.emptyMap; +import static java.util.Collections.singletonList; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; +import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; +import static org.elasticsearch.search.SearchService.DEFAULT_SIZE; +import static org.elasticsearch.search.SearchService.QUERY_PHASE_PARALLEL_COLLECTION_ENABLED; +import static org.elasticsearch.search.SearchService.SEARCH_WORKER_THREADS_ENABLED; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.startsWith; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.not; +import static org.mockito.Mockito.mock; + +public class SearchServiceSingleNodeTests extends ESSingleNodeTestCase { + + @Override + protected boolean resetNodeAfterTest() { + return true; + } + + @Override + protected Collection> getPlugins() { + return pluginList( + FailOnRewriteQueryPlugin.class, + CustomScriptPlugin.class, + ReaderWrapperCountPlugin.class, + InternalOrPrivateSettingsPlugin.class, + MockSearchService.TestPlugin.class + ); + } + + public static class ReaderWrapperCountPlugin extends Plugin { + @Override + public void onIndexModule(IndexModule indexModule) { + indexModule.setReaderWrapper(service -> SearchServiceSingleNodeTests::apply); + } + } + + @Before + public void resetCount() { + numWrapInvocations = new AtomicInteger(0); + } + + private static AtomicInteger numWrapInvocations = new AtomicInteger(0); + + private static DirectoryReader apply(DirectoryReader directoryReader) throws IOException { + numWrapInvocations.incrementAndGet(); + return new FilterDirectoryReader(directoryReader, new FilterDirectoryReader.SubReaderWrapper() { + @Override + public LeafReader wrap(LeafReader reader) { + return reader; + } + }) { + @Override + protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { + return in; + } + + @Override + public CacheHelper getReaderCacheHelper() { + return directoryReader.getReaderCacheHelper(); + } + }; + } + + public static class CustomScriptPlugin extends MockScriptPlugin { + + static final String DUMMY_SCRIPT = "dummyScript"; + + @Override + protected Map, Object>> pluginScripts() { + return Collections.singletonMap(DUMMY_SCRIPT, vars -> "dummy"); + } + + @Override + public void onIndexModule(IndexModule indexModule) { + indexModule.addSearchOperationListener(new SearchOperationListener() { + @Override + public void onFetchPhase(SearchContext context, long tookInNanos) { + if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) { + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search_throttled]")); + } else { + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + } + } + + @Override + public void onQueryPhase(SearchContext context, long tookInNanos) { + if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) { + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search_throttled]")); + } else { + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + } + } + }); + } + } + + @Override + protected Settings nodeSettings() { + return Settings.builder().put("search.default_search_timeout", "5s").build(); + } + + public void testClearOnClose() { + createIndex("index"); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + assertResponse( + client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), + searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) + ); + SearchService service = getInstanceFromNode(SearchService.class); + + assertEquals(1, service.getActiveContexts()); + service.doClose(); // this kills the keep-alive reaper we have to reset the node after this test + assertEquals(0, service.getActiveContexts()); + } + + public void testClearOnStop() { + createIndex("index"); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + assertResponse( + client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), + searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) + ); + SearchService service = getInstanceFromNode(SearchService.class); + + assertEquals(1, service.getActiveContexts()); + service.doStop(); + assertEquals(0, service.getActiveContexts()); + } + + public void testClearIndexDelete() { + createIndex("index"); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + assertResponse( + client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), + searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) + ); + SearchService service = getInstanceFromNode(SearchService.class); + + assertEquals(1, service.getActiveContexts()); + assertAcked(indicesAdmin().prepareDelete("index")); + awaitIndexShardCloseAsyncTasks(); + assertEquals(0, service.getActiveContexts()); + } + + public void testCloseSearchContextOnRewriteException() { + // if refresh happens while checking the exception, the subsequent reference count might not match, so we switch it off + createIndex("index", Settings.builder().put("index.refresh_interval", -1).build()); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + + SearchService service = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + + final int activeContexts = service.getActiveContexts(); + final int activeRefs = indexShard.store().refCount(); + expectThrows( + SearchPhaseExecutionException.class, + () -> client().prepareSearch("index").setQuery(new FailOnRewriteQueryBuilder()).get() + ); + assertEquals(activeContexts, service.getActiveContexts()); + assertEquals(activeRefs, indexShard.store().refCount()); + } + + public void testSearchWhileIndexDeleted() throws InterruptedException { + createIndex("index"); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + + SearchService service = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + AtomicBoolean running = new AtomicBoolean(true); + CountDownLatch startGun = new CountDownLatch(1); + final int permitCount = 100; + Semaphore semaphore = new Semaphore(permitCount); + ShardRouting routing = TestShardRouting.newShardRouting( + indexShard.shardId(), + randomAlphaOfLength(5), + randomBoolean(), + ShardRoutingState.INITIALIZING + ); + final Thread thread = new Thread(() -> { + startGun.countDown(); + while (running.get()) { + if (randomBoolean()) { + service.afterIndexRemoved(indexService.index(), indexService.getIndexSettings(), DELETED); + } else { + service.beforeIndexShardCreated(routing, indexService.getIndexSettings().getSettings()); + } + if (randomBoolean()) { + // here we trigger some refreshes to ensure the IR go out of scope such that we hit ACE if we access a search + // context in a non-sane way. + try { + semaphore.acquire(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + prepareIndex("index").setSource("field", "value") + .setRefreshPolicy(randomFrom(WriteRequest.RefreshPolicy.values())) + .execute(ActionListener.running(semaphore::release)); + } + } + }); + thread.start(); + startGun.await(); + try { + final int rounds = scaledRandomIntBetween(100, 10000); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + SearchRequest scrollSearchRequest = new SearchRequest().allowPartialSearchResults(true) + .scroll(new Scroll(TimeValue.timeValueMinutes(1))); + for (int i = 0; i < rounds; i++) { + try { + try { + PlainActionFuture result = new PlainActionFuture<>(); + final boolean useScroll = randomBoolean(); + service.executeQueryPhase( + new ShardSearchRequest( + OriginalIndices.NONE, + useScroll ? scrollSearchRequest : searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ), + new SearchShardTask(123L, "", "", "", null, emptyMap()), + result.delegateFailure((l, r) -> { + r.incRef(); + l.onResponse(r); + }) + ); + final SearchPhaseResult searchPhaseResult = result.get(); + try { + List intCursors = new ArrayList<>(1); + intCursors.add(0); + ShardFetchRequest req = new ShardFetchRequest( + searchPhaseResult.getContextId(), + intCursors, + null/* not a scroll */ + ); + PlainActionFuture listener = new PlainActionFuture<>(); + service.executeFetchPhase(req, new SearchShardTask(123L, "", "", "", null, emptyMap()), listener); + listener.get(); + if (useScroll) { + // have to free context since this test does not remove the index from IndicesService. + service.freeReaderContext(searchPhaseResult.getContextId()); + } + } finally { + searchPhaseResult.decRef(); + } + } catch (ExecutionException ex) { + assertThat(ex.getCause(), instanceOf(RuntimeException.class)); + throw ((RuntimeException) ex.getCause()); + } + } catch (AlreadyClosedException ex) { + throw ex; + } catch (IllegalStateException ex) { + assertEquals(AbstractRefCounted.ALREADY_CLOSED_MESSAGE, ex.getMessage()); + } catch (SearchContextMissingException ex) { + // that's fine + } + } + } finally { + running.set(false); + thread.join(); + semaphore.acquire(permitCount); + } + + assertEquals(0, service.getActiveContexts()); + + SearchStats.Stats totalStats = indexShard.searchStats().getTotal(); + assertEquals(0, totalStats.getQueryCurrent()); + assertEquals(0, totalStats.getScrollCurrent()); + assertEquals(0, totalStats.getFetchCurrent()); + } + + public void testRankFeaturePhaseSearchPhases() throws InterruptedException, ExecutionException { + final String indexName = "index"; + final String rankFeatureFieldName = "field"; + final String searchFieldName = "search_field"; + final String searchFieldValue = "some_value"; + final String fetchFieldName = "fetch_field"; + final String fetchFieldValue = "fetch_value"; + + final int minDocs = 3; + final int maxDocs = 10; + int numDocs = between(minDocs, maxDocs); + createIndex(indexName); + // index some documents + for (int i = 0; i < numDocs; i++) { + prepareIndex(indexName).setId(String.valueOf(i)) + .setSource( + rankFeatureFieldName, + "aardvark_" + i, + searchFieldName, + searchFieldValue, + fetchFieldName, + fetchFieldValue + "_" + i + ) + .get(); + } + indicesAdmin().prepareRefresh(indexName).get(); + + final SearchService service = getInstanceFromNode(SearchService.class); + + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex(indexName)); + final IndexShard indexShard = indexService.getShard(0); + SearchShardTask searchTask = new SearchShardTask(123L, "", "", "", null, emptyMap()); + + // create a SearchRequest that will return all documents and defines a TestRankBuilder with shard-level only operations + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true) + .source( + new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) + .size(DEFAULT_SIZE) + .fetchField(fetchFieldName) + .rankBuilder( + // here we override only the shard-level contexts + new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + @Override + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { + return new QueryPhaseRankShardContext(queries, from) { + + @Override + public int rankWindowSize() { + return DEFAULT_RANK_WINDOW_SIZE; + } + + @Override + public RankShardResult combineQueryPhaseResults(List rankResults) { + // we know we have just 1 query, so return all the docs from it + return new TestRankShardResult( + Arrays.stream(rankResults.get(0).scoreDocs) + .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) + .limit(rankWindowSize()) + .toArray(RankDoc[]::new) + ); + } + }; + } + + @Override + public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { + return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { + @Override + public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { + RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; + for (int i = 0; i < hits.getHits().length; i++) { + SearchHit hit = hits.getHits()[i]; + rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); + rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); + rankFeatureDocs[i].score = (numDocs - i) + randomFloat(); + rankFeatureDocs[i].rank = i + 1; + } + return new RankFeatureShardResult(rankFeatureDocs); + } + }; + } + } + ) + ); + + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + QuerySearchResult queryResult = null; + RankFeatureResult rankResult = null; + try { + // Execute the query phase and store the result in a SearchPhaseResult container using a PlainActionFuture + PlainActionFuture queryPhaseResults = new PlainActionFuture<>(); + service.executeQueryPhase(request, searchTask, queryPhaseResults); + queryResult = (QuerySearchResult) queryPhaseResults.get(); + + // these are the matched docs from the query phase + final RankDoc[] queryRankDocs = ((TestRankShardResult) queryResult.getRankShardResult()).testRankDocs; + + // assume that we have cut down to these from the coordinator node as the top-docs to run the rank feature phase upon + List topRankWindowSizeDocs = randomNonEmptySubsetOf(Arrays.stream(queryRankDocs).map(x -> x.doc).toList()); + + // now we create a RankFeatureShardRequest to extract feature info for the top-docs above + RankFeatureShardRequest rankFeatureShardRequest = new RankFeatureShardRequest( + OriginalIndices.NONE, + queryResult.getContextId(), // use the context from the query phase + request, + topRankWindowSizeDocs + ); + PlainActionFuture rankPhaseResults = new PlainActionFuture<>(); + service.executeRankFeaturePhase(rankFeatureShardRequest, searchTask, rankPhaseResults); + rankResult = rankPhaseResults.get(); + + assertNotNull(rankResult); + assertNotNull(rankResult.rankFeatureResult()); + RankFeatureShardResult rankFeatureShardResult = rankResult.rankFeatureResult().shardResult(); + assertNotNull(rankFeatureShardResult); + + List sortedRankWindowDocs = topRankWindowSizeDocs.stream().sorted().toList(); + assertEquals(sortedRankWindowDocs.size(), rankFeatureShardResult.rankFeatureDocs.length); + for (int i = 0; i < sortedRankWindowDocs.size(); i++) { + assertEquals((long) sortedRankWindowDocs.get(i), rankFeatureShardResult.rankFeatureDocs[i].doc); + assertEquals(rankFeatureShardResult.rankFeatureDocs[i].featureData, "aardvark_" + sortedRankWindowDocs.get(i)); + } + + List globalTopKResults = randomNonEmptySubsetOf( + Arrays.stream(rankFeatureShardResult.rankFeatureDocs).map(x -> x.doc).toList() + ); + + // finally let's create a fetch request to bring back fetch info for the top results + ShardFetchSearchRequest fetchRequest = new ShardFetchSearchRequest( + OriginalIndices.NONE, + rankResult.getContextId(), + request, + globalTopKResults, + null, + null, + rankResult.getRescoreDocIds(), + null + ); + + // execute fetch phase and perform any validations once we retrieve the response + // the difference in how we do assertions here is needed because once the transport service sends back the response + // it decrements the reference to the FetchSearchResult (through the ActionListener#respondAndRelease) and sets hits to null + PlainActionFuture fetchListener = new PlainActionFuture<>() { + @Override + public void onResponse(FetchSearchResult fetchSearchResult) { + assertNotNull(fetchSearchResult); + assertNotNull(fetchSearchResult.hits()); + + int totalHits = fetchSearchResult.hits().getHits().length; + assertEquals(globalTopKResults.size(), totalHits); + for (int i = 0; i < totalHits; i++) { + // rank and score are set by the SearchPhaseController#merge so no need to validate that here + SearchHit hit = fetchSearchResult.hits().getAt(i); + assertNotNull(hit.getFields().get(fetchFieldName)); + assertEquals(hit.getFields().get(fetchFieldName).getValue(), fetchFieldValue + "_" + hit.docId()); + } + super.onResponse(fetchSearchResult); + } + + @Override + public void onFailure(Exception e) { + super.onFailure(e); + throw new AssertionError("No failure should have been raised", e); + } + }; + service.executeFetchPhase(fetchRequest, searchTask, fetchListener); + fetchListener.get(); + } catch (Exception ex) { + if (queryResult != null) { + if (queryResult.hasReferences()) { + queryResult.decRef(); + } + service.freeReaderContext(queryResult.getContextId()); + } + if (rankResult != null && rankResult.hasReferences()) { + rankResult.decRef(); + } + throw ex; + } + } + + public void testRankFeaturePhaseUsingClient() { + final String indexName = "index"; + final String rankFeatureFieldName = "field"; + final String searchFieldName = "search_field"; + final String searchFieldValue = "some_value"; + final String fetchFieldName = "fetch_field"; + final String fetchFieldValue = "fetch_value"; + + final int minDocs = 4; + final int maxDocs = 10; + int numDocs = between(minDocs, maxDocs); + createIndex(indexName); + // index some documents + for (int i = 0; i < numDocs; i++) { + prepareIndex(indexName).setId(String.valueOf(i)) + .setSource( + rankFeatureFieldName, + "aardvark_" + i, + searchFieldName, + searchFieldValue, + fetchFieldName, + fetchFieldValue + "_" + i + ) + .get(); + } + indicesAdmin().prepareRefresh(indexName).get(); + + ElasticsearchAssertions.assertResponse( + client().prepareSearch(indexName) + .setSource( + new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) + .size(2) + .from(2) + .fetchField(fetchFieldName) + .rankBuilder( + // here we override only the shard-level contexts + new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + + // no need for more than one queries + @Override + public boolean isCompoundBuilder() { + return false; + } + + @Override + public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( + int size, + int from, + Client client + ) { + return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { + @Override + protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { + float[] scores = new float[featureDocs.length]; + for (int i = 0; i < featureDocs.length; i++) { + scores[i] = featureDocs[i].score; + } + scoreListener.onResponse(scores); + } + }; + } + + @Override + public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { + return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + @Override + public ScoreDoc[] rankQueryPhaseResults( + List querySearchResults, + SearchPhaseController.TopDocsStats topDocStats + ) { + List rankDocs = new ArrayList<>(); + for (int i = 0; i < querySearchResults.size(); i++) { + QuerySearchResult querySearchResult = querySearchResults.get(i); + TestRankShardResult shardResult = (TestRankShardResult) querySearchResult + .getRankShardResult(); + for (RankDoc trd : shardResult.testRankDocs) { + trd.shardIndex = i; + rankDocs.add(trd); + } + } + rankDocs.sort(Comparator.comparing((RankDoc doc) -> doc.score).reversed()); + RankDoc[] topResults = rankDocs.stream().limit(rankWindowSize).toArray(RankDoc[]::new); + topDocStats.fetchHits = topResults.length; + return topResults; + } + }; + } + + @Override + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { + return new QueryPhaseRankShardContext(queries, from) { + + @Override + public int rankWindowSize() { + return DEFAULT_RANK_WINDOW_SIZE; + } + + @Override + public RankShardResult combineQueryPhaseResults(List rankResults) { + // we know we have just 1 query, so return all the docs from it + return new TestRankShardResult( + Arrays.stream(rankResults.get(0).scoreDocs) + .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) + .limit(rankWindowSize()) + .toArray(RankDoc[]::new) + ); + } + }; + } + + @Override + public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { + return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { + @Override + public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { + RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; + for (int i = 0; i < hits.getHits().length; i++) { + SearchHit hit = hits.getHits()[i]; + rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); + rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); + rankFeatureDocs[i].score = randomFloat(); + rankFeatureDocs[i].rank = i + 1; + } + return new RankFeatureShardResult(rankFeatureDocs); + } + }; + } + } + ) + ), + (response) -> { + SearchHits hits = response.getHits(); + assertEquals(hits.getTotalHits().value, numDocs); + assertEquals(hits.getHits().length, 2); + int index = 0; + for (SearchHit hit : hits.getHits()) { + assertEquals(hit.getRank(), 3 + index); + assertTrue(hit.getScore() >= 0); + assertEquals(hit.getFields().get(fetchFieldName).getValue(), fetchFieldValue + "_" + hit.docId()); + index++; + } + } + ); + } + + public void testRankFeaturePhaseExceptionOnCoordinatingNode() { + final String indexName = "index"; + final String rankFeatureFieldName = "field"; + final String searchFieldName = "search_field"; + final String searchFieldValue = "some_value"; + final String fetchFieldName = "fetch_field"; + final String fetchFieldValue = "fetch_value"; + + final int minDocs = 3; + final int maxDocs = 10; + int numDocs = between(minDocs, maxDocs); + createIndex(indexName); + // index some documents + for (int i = 0; i < numDocs; i++) { + prepareIndex(indexName).setId(String.valueOf(i)) + .setSource( + rankFeatureFieldName, + "aardvark_" + i, + searchFieldName, + searchFieldValue, + fetchFieldName, + fetchFieldValue + "_" + i + ) + .get(); + } + indicesAdmin().prepareRefresh(indexName).get(); + + expectThrows( + SearchPhaseExecutionException.class, + () -> client().prepareSearch(indexName) + .setSource( + new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) + .size(2) + .from(2) + .fetchField(fetchFieldName) + .rankBuilder(new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + + // no need for more than one queries + @Override + public boolean isCompoundBuilder() { + return false; + } + + @Override + public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( + int size, + int from, + Client client + ) { + return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { + @Override + protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { + throw new IllegalStateException("should have failed earlier"); + } + }; + } + + @Override + public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { + return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + @Override + public ScoreDoc[] rankQueryPhaseResults( + List querySearchResults, + SearchPhaseController.TopDocsStats topDocStats + ) { + throw new UnsupportedOperationException("simulated failure"); + } + }; + } + + @Override + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { + return new QueryPhaseRankShardContext(queries, from) { + + @Override + public int rankWindowSize() { + return DEFAULT_RANK_WINDOW_SIZE; + } + + @Override + public RankShardResult combineQueryPhaseResults(List rankResults) { + // we know we have just 1 query, so return all the docs from it + return new TestRankShardResult( + Arrays.stream(rankResults.get(0).scoreDocs) + .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) + .limit(rankWindowSize()) + .toArray(RankDoc[]::new) + ); + } + }; + } + + @Override + public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { + return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { + @Override + public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { + RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; + for (int i = 0; i < hits.getHits().length; i++) { + SearchHit hit = hits.getHits()[i]; + rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); + rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); + rankFeatureDocs[i].score = randomFloat(); + rankFeatureDocs[i].rank = i + 1; + } + return new RankFeatureShardResult(rankFeatureDocs); + } + }; + } + }) + ) + .get() + ); + } + + public void testRankFeaturePhaseExceptionAllShardFail() { + final String indexName = "index"; + final String rankFeatureFieldName = "field"; + final String searchFieldName = "search_field"; + final String searchFieldValue = "some_value"; + final String fetchFieldName = "fetch_field"; + final String fetchFieldValue = "fetch_value"; + + final int minDocs = 3; + final int maxDocs = 10; + int numDocs = between(minDocs, maxDocs); + createIndex(indexName); + // index some documents + for (int i = 0; i < numDocs; i++) { + prepareIndex(indexName).setId(String.valueOf(i)) + .setSource( + rankFeatureFieldName, + "aardvark_" + i, + searchFieldName, + searchFieldValue, + fetchFieldName, + fetchFieldValue + "_" + i + ) + .get(); + } + indicesAdmin().prepareRefresh(indexName).get(); + + expectThrows( + SearchPhaseExecutionException.class, + () -> client().prepareSearch(indexName) + .setAllowPartialSearchResults(true) + .setSource( + new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) + .fetchField(fetchFieldName) + .rankBuilder( + // here we override only the shard-level contexts + new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + + // no need for more than one queries + @Override + public boolean isCompoundBuilder() { + return false; + } + + @Override + public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( + int size, + int from, + Client client + ) { + return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { + @Override + protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { + float[] scores = new float[featureDocs.length]; + for (int i = 0; i < featureDocs.length; i++) { + scores[i] = featureDocs[i].score; + } + scoreListener.onResponse(scores); + } + }; + } + + @Override + public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { + return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + @Override + public ScoreDoc[] rankQueryPhaseResults( + List querySearchResults, + SearchPhaseController.TopDocsStats topDocStats + ) { + List rankDocs = new ArrayList<>(); + for (int i = 0; i < querySearchResults.size(); i++) { + QuerySearchResult querySearchResult = querySearchResults.get(i); + TestRankShardResult shardResult = (TestRankShardResult) querySearchResult + .getRankShardResult(); + for (RankDoc trd : shardResult.testRankDocs) { + trd.shardIndex = i; + rankDocs.add(trd); + } + } + rankDocs.sort(Comparator.comparing((RankDoc doc) -> doc.score).reversed()); + RankDoc[] topResults = rankDocs.stream().limit(rankWindowSize).toArray(RankDoc[]::new); + topDocStats.fetchHits = topResults.length; + return topResults; + } + }; + } + + @Override + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { + return new QueryPhaseRankShardContext(queries, from) { + + @Override + public int rankWindowSize() { + return DEFAULT_RANK_WINDOW_SIZE; + } + + @Override + public RankShardResult combineQueryPhaseResults(List rankResults) { + // we know we have just 1 query, so return all the docs from it + return new TestRankShardResult( + Arrays.stream(rankResults.get(0).scoreDocs) + .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) + .limit(rankWindowSize()) + .toArray(RankDoc[]::new) + ); + } + }; + } + + @Override + public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { + return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { + @Override + public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { + throw new UnsupportedOperationException("simulated failure"); + } + }; + } + } + ) + ) + .get() + ); + } + + public void testRankFeaturePhaseExceptionOneShardFails() { + // if we have only one shard and it fails, it will fallback to context.onPhaseFailure which will eventually clean up all contexts. + // in this test we want to make sure that even if one shard (of many) fails during the RankFeaturePhase, then the appropriate + // context will have been cleaned up. + final String indexName = "index"; + final String rankFeatureFieldName = "field"; + final String searchFieldName = "search_field"; + final String searchFieldValue = "some_value"; + final String fetchFieldName = "fetch_field"; + final String fetchFieldValue = "fetch_value"; + + final int minDocs = 3; + final int maxDocs = 10; + int numDocs = between(minDocs, maxDocs); + createIndex(indexName, Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2).build()); + // index some documents + for (int i = 0; i < numDocs; i++) { + prepareIndex(indexName).setId(String.valueOf(i)) + .setSource( + rankFeatureFieldName, + "aardvark_" + i, + searchFieldName, + searchFieldValue, + fetchFieldName, + fetchFieldValue + "_" + i + ) + .get(); + } + indicesAdmin().prepareRefresh(indexName).get(); + + assertResponse( + client().prepareSearch(indexName) + .setAllowPartialSearchResults(true) + .setSource( + new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) + .fetchField(fetchFieldName) + .rankBuilder( + // here we override only the shard-level contexts + new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + + // no need for more than one queries + @Override + public boolean isCompoundBuilder() { + return false; + } + + @Override + public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( + int size, + int from, + Client client + ) { + return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { + @Override + protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { + float[] scores = new float[featureDocs.length]; + for (int i = 0; i < featureDocs.length; i++) { + scores[i] = featureDocs[i].score; + } + scoreListener.onResponse(scores); + } + }; + } + + @Override + public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { + return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { + @Override + public ScoreDoc[] rankQueryPhaseResults( + List querySearchResults, + SearchPhaseController.TopDocsStats topDocStats + ) { + List rankDocs = new ArrayList<>(); + for (int i = 0; i < querySearchResults.size(); i++) { + QuerySearchResult querySearchResult = querySearchResults.get(i); + TestRankShardResult shardResult = (TestRankShardResult) querySearchResult + .getRankShardResult(); + for (RankDoc trd : shardResult.testRankDocs) { + trd.shardIndex = i; + rankDocs.add(trd); + } + } + rankDocs.sort(Comparator.comparing((RankDoc doc) -> doc.score).reversed()); + RankDoc[] topResults = rankDocs.stream().limit(rankWindowSize).toArray(RankDoc[]::new); + topDocStats.fetchHits = topResults.length; + return topResults; + } + }; + } + + @Override + public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { + return new QueryPhaseRankShardContext(queries, from) { + + @Override + public int rankWindowSize() { + return DEFAULT_RANK_WINDOW_SIZE; + } + + @Override + public RankShardResult combineQueryPhaseResults(List rankResults) { + // we know we have just 1 query, so return all the docs from it + return new TestRankShardResult( + Arrays.stream(rankResults.get(0).scoreDocs) + .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) + .limit(rankWindowSize()) + .toArray(RankDoc[]::new) + ); + } + }; + } + + @Override + public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { + return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { + @Override + public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { + if (shardId == 0) { + throw new UnsupportedOperationException("simulated failure"); + } else { + RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; + for (int i = 0; i < hits.getHits().length; i++) { + SearchHit hit = hits.getHits()[i]; + rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); + rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); + rankFeatureDocs[i].score = randomFloat(); + rankFeatureDocs[i].rank = i + 1; + } + return new RankFeatureShardResult(rankFeatureDocs); + } + } + }; + } + } + ) + ), + (searchResponse) -> { + assertEquals(1, searchResponse.getSuccessfulShards()); + assertEquals("simulated failure", searchResponse.getShardFailures()[0].getCause().getMessage()); + assertNotEquals(0, searchResponse.getHits().getHits().length); + for (SearchHit hit : searchResponse.getHits().getHits()) { + assertEquals(fetchFieldValue + "_" + hit.getId(), hit.getFields().get(fetchFieldName).getValue()); + assertEquals(1, hit.getShard().getShardId().id()); + } + } + ); + } + + public void testSearchWhileIndexDeletedDoesNotLeakSearchContext() throws ExecutionException, InterruptedException { + createIndex("index"); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + + MockSearchService service = (MockSearchService) getInstanceFromNode(SearchService.class); + service.setOnPutContext(context -> { + if (context.indexShard() == indexShard) { + assertAcked(indicesAdmin().prepareDelete("index")); + } + }); + + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + SearchRequest scrollSearchRequest = new SearchRequest().allowPartialSearchResults(true) + .scroll(new Scroll(TimeValue.timeValueMinutes(1))); + + // the scrolls are not explicitly freed, but should all be gone when the test finished. + // for completeness, we also randomly test the regular search path. + final boolean useScroll = randomBoolean(); + PlainActionFuture result = new PlainActionFuture<>(); + service.executeQueryPhase( + new ShardSearchRequest( + OriginalIndices.NONE, + useScroll ? scrollSearchRequest : searchRequest, + new ShardId(resolveIndex("index"), 0), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ), + new SearchShardTask(123L, "", "", "", null, emptyMap()), + result + ); + + try { + result.get(); + } catch (Exception e) { + // ok + } + + expectThrows(IndexNotFoundException.class, () -> indicesAdmin().prepareGetIndex().setIndices("index").get()); + + assertEquals(0, service.getActiveContexts()); + + SearchStats.Stats totalStats = indexShard.searchStats().getTotal(); + assertEquals(0, totalStats.getQueryCurrent()); + assertEquals(0, totalStats.getScrollCurrent()); + assertEquals(0, totalStats.getFetchCurrent()); + } + + public void testBeforeShardLockDuringShardCreate() { + IndexService indexService = createIndex("index", Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build()); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + assertResponse( + client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), + searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) + ); + SearchService service = getInstanceFromNode(SearchService.class); + + assertEquals(1, service.getActiveContexts()); + service.beforeIndexShardCreated( + TestShardRouting.newShardRouting( + "test", + 0, + randomAlphaOfLength(5), + randomAlphaOfLength(5), + randomBoolean(), + ShardRoutingState.INITIALIZING + ), + indexService.getIndexSettings().getSettings() + ); + assertEquals(1, service.getActiveContexts()); + + service.beforeIndexShardCreated( + TestShardRouting.newShardRouting( + new ShardId(indexService.index(), 0), + randomAlphaOfLength(5), + randomBoolean(), + ShardRoutingState.INITIALIZING + ), + indexService.getIndexSettings().getSettings() + ); + assertEquals(0, service.getActiveContexts()); + } + + public void testTimeout() throws IOException { + createIndex("index"); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + final ShardSearchRequest requestWithDefaultTimeout = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + + try ( + ReaderContext reader = createReaderContext(indexService, indexShard); + SearchContext contextWithDefaultTimeout = service.createContext( + reader, + requestWithDefaultTimeout, + mock(SearchShardTask.class), + ResultsType.NONE, + randomBoolean() + ) + ) { + // the search context should inherit the default timeout + assertThat(contextWithDefaultTimeout.timeout(), equalTo(TimeValue.timeValueSeconds(5))); + } + + final long seconds = randomIntBetween(6, 10); + searchRequest.source(new SearchSourceBuilder().timeout(TimeValue.timeValueSeconds(seconds))); + final ShardSearchRequest requestWithCustomTimeout = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + try ( + ReaderContext reader = createReaderContext(indexService, indexShard); + SearchContext context = service.createContext( + reader, + requestWithCustomTimeout, + mock(SearchShardTask.class), + ResultsType.NONE, + randomBoolean() + ) + ) { + // the search context should inherit the query timeout + assertThat(context.timeout(), equalTo(TimeValue.timeValueSeconds(seconds))); + } + } + + /** + * test that getting more than the allowed number of docvalue_fields throws an exception + */ + public void testMaxDocvalueFieldsSearch() throws IOException { + final Settings settings = Settings.builder().put(IndexSettings.MAX_DOCVALUE_FIELDS_SEARCH_SETTING.getKey(), 1).build(); + createIndex("index", settings, null, "field1", "keyword", "field2", "keyword"); + prepareIndex("index").setId("1").setSource("field1", "value1", "field2", "value2").setRefreshPolicy(IMMEDIATE).get(); + + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchRequest.source(searchSourceBuilder); + searchSourceBuilder.docValueField("field1"); + + final ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + try ( + ReaderContext reader = createReaderContext(indexService, indexShard); + SearchContext context = service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) + ) { + assertNotNull(context); + } + + searchSourceBuilder.docValueField("unmapped_field"); + try ( + ReaderContext reader = createReaderContext(indexService, indexShard); + SearchContext context = service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) + ) { + assertNotNull(context); + } + + searchSourceBuilder.docValueField("field2"); + try (ReaderContext reader = createReaderContext(indexService, indexShard)) { + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) + ); + assertEquals( + "Trying to retrieve too many docvalue_fields. Must be less than or equal to: [1] but was [2]. " + + "This limit can be set by changing the [index.max_docvalue_fields_search] index level setting.", + ex.getMessage() + ); + } + } + + public void testDeduplicateDocValuesFields() throws Exception { + createIndex("index", Settings.EMPTY, "_doc", "field1", "type=date", "field2", "type=date"); + prepareIndex("index").setId("1").setSource("field1", "2022-08-03", "field2", "2022-08-04").setRefreshPolicy(IMMEDIATE).get(); + SearchService service = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + + try (ReaderContext reader = createReaderContext(indexService, indexShard)) { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchRequest.source(searchSourceBuilder); + searchSourceBuilder.docValueField("f*"); + if (randomBoolean()) { + searchSourceBuilder.docValueField("field*"); + } + if (randomBoolean()) { + searchSourceBuilder.docValueField("*2"); + } + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + try ( + SearchContext context = service.createContext( + reader, + request, + mock(SearchShardTask.class), + ResultsType.NONE, + randomBoolean() + ) + ) { + Collection fields = context.docValuesContext().fields(); + assertThat(fields, containsInAnyOrder(new FieldAndFormat("field1", null), new FieldAndFormat("field2", null))); + } + } + } + + /** + * test that getting more than the allowed number of script_fields throws an exception + */ + public void testMaxScriptFieldsSearch() throws IOException { + createIndex("index"); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchRequest.source(searchSourceBuilder); + // adding the maximum allowed number of script_fields to retrieve + int maxScriptFields = indexService.getIndexSettings().getMaxScriptFields(); + for (int i = 0; i < maxScriptFields; i++) { + searchSourceBuilder.scriptField( + "field" + i, + new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, emptyMap()) + ); + } + final ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + + try (ReaderContext reader = createReaderContext(indexService, indexShard)) { + try ( + SearchContext context = service.createContext( + reader, + request, + mock(SearchShardTask.class), + ResultsType.NONE, + randomBoolean() + ) + ) { + assertNotNull(context); + } + searchSourceBuilder.scriptField( + "anotherScriptField", + new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, emptyMap()) + ); + IllegalArgumentException ex = expectThrows( + IllegalArgumentException.class, + () -> service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) + ); + assertEquals( + "Trying to retrieve too many script_fields. Must be less than or equal to: [" + + maxScriptFields + + "] but was [" + + (maxScriptFields + 1) + + "]. This limit can be set by changing the [index.max_script_fields] index level setting.", + ex.getMessage() + ); + } + } + + public void testIgnoreScriptfieldIfSizeZero() throws IOException { + createIndex("index"); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); + searchRequest.source(searchSourceBuilder); + searchSourceBuilder.scriptField( + "field" + 0, + new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, emptyMap()) + ); + searchSourceBuilder.size(0); + final ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + try ( + ReaderContext reader = createReaderContext(indexService, indexShard); + SearchContext context = service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) + ) { + assertEquals(0, context.scriptFields().fields().size()); + } + } + + /** + * test that creating more than the allowed number of scroll contexts throws an exception + */ + public void testMaxOpenScrollContexts() throws Exception { + createIndex("index"); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + + // Open all possible scrolls, clear some of them, then open more until the limit is reached + LinkedList clearScrollIds = new LinkedList<>(); + + for (int i = 0; i < SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY); i++) { + assertResponse(client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), searchResponse -> { + if (randomInt(4) == 0) clearScrollIds.addLast(searchResponse.getScrollId()); + }); + } + + ClearScrollRequest clearScrollRequest = new ClearScrollRequest(); + clearScrollRequest.setScrollIds(clearScrollIds); + client().clearScroll(clearScrollRequest).get(); + + for (int i = 0; i < clearScrollIds.size(); i++) { + client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)).get().decRef(); + } + + final ShardScrollRequestTest request = new ShardScrollRequestTest(indexShard.shardId()); + ElasticsearchException ex = expectThrows( + ElasticsearchException.class, + () -> service.createAndPutReaderContext( + request, + indexService, + indexShard, + indexShard.acquireSearcherSupplier(), + SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() + ) + ); + assertEquals( + "Trying to create too many scroll contexts. Must be less than or equal to: [" + + SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY) + + "]. " + + "This limit can be set by changing the [search.max_open_scroll_context] setting.", + ex.getMessage() + ); + assertEquals(RestStatus.TOO_MANY_REQUESTS, ex.status()); + + service.freeAllScrollContexts(); + } + + public void testOpenScrollContextsConcurrently() throws Exception { + createIndex("index"); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + + final int maxScrollContexts = SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY); + final SearchService searchService = getInstanceFromNode(SearchService.class); + Thread[] threads = new Thread[randomIntBetween(2, 8)]; + CountDownLatch latch = new CountDownLatch(threads.length); + for (int i = 0; i < threads.length; i++) { + threads[i] = new Thread(() -> { + latch.countDown(); + try { + latch.await(); + for (;;) { + final Engine.SearcherSupplier reader = indexShard.acquireSearcherSupplier(); + try { + final ShardScrollRequestTest request = new ShardScrollRequestTest(indexShard.shardId()); + searchService.createAndPutReaderContext( + request, + indexService, + indexShard, + reader, + SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() + ); + } catch (ElasticsearchException e) { + assertThat( + e.getMessage(), + equalTo( + "Trying to create too many scroll contexts. Must be less than or equal to: " + + "[" + + maxScrollContexts + + "]. " + + "This limit can be set by changing the [search.max_open_scroll_context] setting." + ) + ); + return; + } + } + } catch (Exception e) { + throw new AssertionError(e); + } + }); + threads[i].setName("elasticsearch[node_s_0][search]"); + threads[i].start(); + } + for (Thread thread : threads) { + thread.join(); + } + assertThat(searchService.getActiveContexts(), equalTo(maxScrollContexts)); + searchService.freeAllScrollContexts(); + } + + public static class FailOnRewriteQueryPlugin extends Plugin implements SearchPlugin { + @Override + public List> getQueries() { + return singletonList(new QuerySpec<>("fail_on_rewrite_query", FailOnRewriteQueryBuilder::new, parseContext -> { + throw new UnsupportedOperationException("No query parser for this plugin"); + })); + } + } + + public static class FailOnRewriteQueryBuilder extends DummyQueryBuilder { + + public FailOnRewriteQueryBuilder(StreamInput in) throws IOException { + super(in); + } + + public FailOnRewriteQueryBuilder() {} + + @Override + protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) { + if (queryRewriteContext.convertToSearchExecutionContext() != null) { + throw new IllegalStateException("Fail on rewrite phase"); + } + return this; + } + } + + private static class ShardScrollRequestTest extends ShardSearchRequest { + private Scroll scroll; + + ShardScrollRequestTest(ShardId shardId) { + super( + OriginalIndices.NONE, + new SearchRequest().allowPartialSearchResults(true), + shardId, + 0, + 1, + AliasFilter.EMPTY, + 1f, + -1, + null + ); + this.scroll = new Scroll(TimeValue.timeValueMinutes(1)); + } + + @Override + public Scroll scroll() { + return this.scroll; + } + } + + public void testCanMatch() throws Exception { + createIndex("index"); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + assertTrue( + service.canMatch( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) + ).canMatch() + ); + + searchRequest.source(new SearchSourceBuilder()); + assertTrue( + service.canMatch( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) + ).canMatch() + ); + + searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder())); + assertTrue( + service.canMatch( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) + ).canMatch() + ); + + searchRequest.source( + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(0)) + ); + assertTrue( + service.canMatch( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) + ).canMatch() + ); + searchRequest.source( + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()).aggregation(new GlobalAggregationBuilder("test")) + ); + assertTrue( + service.canMatch( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) + ).canMatch() + ); + + searchRequest.source(new SearchSourceBuilder().query(new MatchNoneQueryBuilder())); + assertFalse( + service.canMatch( + new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) + ).canMatch() + ); + assertEquals(5, numWrapInvocations.get()); + + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + + /* + * Checks that canMatch takes into account the alias filter + */ + // the source cannot be rewritten to a match_none + searchRequest.indices("alias").source(new SearchSourceBuilder().query(new MatchAllQueryBuilder())); + assertFalse( + service.canMatch( + new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.of(new TermQueryBuilder("foo", "bar"), "alias"), + 1f, + -1, + null + ) + ).canMatch() + ); + // the source can match and can be rewritten to a match_none, but not the alias filter + final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); + assertEquals(RestStatus.CREATED, response.status()); + searchRequest.indices("alias").source(new SearchSourceBuilder().query(new TermQueryBuilder("id", "1"))); + assertFalse( + service.canMatch( + new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.of(new TermQueryBuilder("foo", "bar"), "alias"), + 1f, + -1, + null + ) + ).canMatch() + ); + + CountDownLatch latch = new CountDownLatch(1); + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); + // Because the foo field used in alias filter is unmapped the term query builder rewrite can resolve to a match no docs query, + // without acquiring a searcher and that means the wrapper is not called + assertEquals(5, numWrapInvocations.get()); + service.executeQueryPhase(request, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + try { + // make sure that the wrapper is called when the query is actually executed + assertEquals(6, numWrapInvocations.get()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception e) { + try { + throw new AssertionError(e); + } finally { + latch.countDown(); + } + } + }); + latch.await(); + } + + public void testCanRewriteToMatchNone() { + assertFalse( + SearchService.canRewriteToMatchNone( + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()).aggregation(new GlobalAggregationBuilder("test")) + ) + ); + assertFalse(SearchService.canRewriteToMatchNone(new SearchSourceBuilder())); + assertFalse(SearchService.canRewriteToMatchNone(null)); + assertFalse( + SearchService.canRewriteToMatchNone( + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(0)) + ) + ); + assertTrue(SearchService.canRewriteToMatchNone(new SearchSourceBuilder().query(new TermQueryBuilder("foo", "bar")))); + assertTrue( + SearchService.canRewriteToMatchNone( + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(1)) + ) + ); + assertFalse( + SearchService.canRewriteToMatchNone( + new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) + .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(1)) + .suggest(new SuggestBuilder()) + ) + ); + assertFalse( + SearchService.canRewriteToMatchNone( + new SearchSourceBuilder().query(new TermQueryBuilder("foo", "bar")).suggest(new SuggestBuilder()) + ) + ); + } + + public void testSetSearchThrottled() throws IOException { + createIndex("throttled_threadpool_index"); + client().execute( + InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, + new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( + "throttled_threadpool_index", + IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), + "true" + ) + ).actionGet(); + final SearchService service = getInstanceFromNode(SearchService.class); + Index index = resolveIndex("throttled_threadpool_index"); + assertTrue(service.getIndicesService().indexServiceSafe(index).getIndexSettings().isSearchThrottled()); + prepareIndex("throttled_threadpool_index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + assertSearchHits( + client().prepareSearch("throttled_threadpool_index") + .setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED) + .setSize(1), + "1" + ); + // we add a search action listener in a plugin above to assert that this is actually used + client().execute( + InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, + new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( + "throttled_threadpool_index", + IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), + "false" + ) + ).actionGet(); + + IllegalArgumentException iae = expectThrows( + IllegalArgumentException.class, + () -> indicesAdmin().prepareUpdateSettings("throttled_threadpool_index") + .setSettings(Settings.builder().put(IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), false)) + .get() + ); + assertEquals("can not update private setting [index.search.throttled]; this setting is managed by Elasticsearch", iae.getMessage()); + assertFalse(service.getIndicesService().indexServiceSafe(index).getIndexSettings().isSearchThrottled()); + } + + public void testAggContextGetsMatchAll() throws IOException { + createIndex("test"); + withAggregationContext("test", context -> assertThat(context.query(), equalTo(new MatchAllDocsQuery()))); + } + + public void testAggContextGetsNestedFilter() throws IOException { + XContentBuilder mapping = JsonXContent.contentBuilder().startObject().startObject("properties"); + mapping.startObject("nested").field("type", "nested").endObject(); + mapping.endObject().endObject(); + + createIndex("test", Settings.EMPTY, mapping); + withAggregationContext("test", context -> assertThat(context.query(), equalTo(new MatchAllDocsQuery()))); + } + + /** + * Build an {@link AggregationContext} with the named index. + */ + private void withAggregationContext(String index, Consumer check) throws IOException { + IndexService indexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(resolveIndex(index)); + ShardId shardId = new ShardId(indexService.index(), 0); + + SearchRequest request = new SearchRequest().indices(index) + .source(new SearchSourceBuilder().aggregation(new FiltersAggregationBuilder("test", new MatchAllQueryBuilder()))) + .allowPartialSearchResults(false); + ShardSearchRequest shardRequest = new ShardSearchRequest( + OriginalIndices.NONE, + request, + shardId, + 0, + 1, + AliasFilter.EMPTY, + 1, + 0, + null + ); + + try (ReaderContext readerContext = createReaderContext(indexService, indexService.getShard(0))) { + try ( + SearchContext context = getInstanceFromNode(SearchService.class).createContext( + readerContext, + shardRequest, + mock(SearchShardTask.class), + ResultsType.QUERY, + true + ) + ) { + check.accept(context.aggregations().factories().context()); + } + } + } + + public void testExpandSearchThrottled() { + createIndex("throttled_threadpool_index"); + client().execute( + InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, + new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( + "throttled_threadpool_index", + IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), + "true" + ) + ).actionGet(); + + prepareIndex("throttled_threadpool_index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + assertHitCount(client().prepareSearch(), 1L); + assertHitCount(client().prepareSearch().setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED), 1L); + } + + public void testExpandSearchFrozen() { + String indexName = "frozen_index"; + createIndex(indexName); + client().execute( + InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, + new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request(indexName, "index.frozen", "true") + ).actionGet(); + + prepareIndex(indexName).setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + assertHitCount(client().prepareSearch(), 0L); + assertHitCount(client().prepareSearch().setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED), 1L); + assertWarnings(TransportSearchAction.FROZEN_INDICES_DEPRECATION_MESSAGE.replace("{}", indexName)); + } + + public void testCreateReduceContext() { + SearchService service = getInstanceFromNode(SearchService.class); + AggregationReduceContext.Builder reduceContextBuilder = service.aggReduceContextBuilder( + () -> false, + new SearchRequest().source(new SearchSourceBuilder()).source().aggregations() + ); + { + AggregationReduceContext reduceContext = reduceContextBuilder.forFinalReduction(); + expectThrows( + MultiBucketConsumerService.TooManyBucketsException.class, + () -> reduceContext.consumeBucketsAndMaybeBreak(MultiBucketConsumerService.DEFAULT_MAX_BUCKETS + 1) + ); + } + { + AggregationReduceContext reduceContext = reduceContextBuilder.forPartialReduction(); + reduceContext.consumeBucketsAndMaybeBreak(MultiBucketConsumerService.DEFAULT_MAX_BUCKETS + 1); + } + } + + public void testMultiBucketConsumerServiceCB() { + MultiBucketConsumerService service = new MultiBucketConsumerService( + getInstanceFromNode(ClusterService.class), + Settings.EMPTY, + new NoopCircuitBreaker("test") { + + @Override + public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException { + throw new CircuitBreakingException("tripped", getDurability()); + } + } + ); + // for partial + { + IntConsumer consumer = service.createForPartial(); + for (int i = 0; i < 1023; i++) { + consumer.accept(0); + } + CircuitBreakingException ex = expectThrows(CircuitBreakingException.class, () -> consumer.accept(0)); + assertThat(ex.getMessage(), equalTo("tripped")); + } + // for final + { + IntConsumer consumer = service.createForFinal(); + for (int i = 0; i < 1023; i++) { + consumer.accept(0); + } + CircuitBreakingException ex = expectThrows(CircuitBreakingException.class, () -> consumer.accept(0)); + assertThat(ex.getMessage(), equalTo("tripped")); + } + } + + public void testCreateSearchContext() throws IOException { + String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); + IndexService indexService = createIndex(index); + final SearchService service = getInstanceFromNode(SearchService.class); + ShardId shardId = new ShardId(indexService.index(), 0); + long nowInMillis = System.currentTimeMillis(); + String clusterAlias = randomBoolean() ? null : randomAlphaOfLengthBetween(3, 10); + SearchRequest searchRequest = new SearchRequest(); + searchRequest.allowPartialSearchResults(randomBoolean()); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + shardId, + 0, + indexService.numberOfShards(), + AliasFilter.EMPTY, + 1f, + nowInMillis, + clusterAlias + ); + try (SearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()))) { + SearchShardTarget searchShardTarget = searchContext.shardTarget(); + SearchExecutionContext searchExecutionContext = searchContext.getSearchExecutionContext(); + String expectedIndexName = clusterAlias == null ? index : clusterAlias + ":" + index; + assertEquals(expectedIndexName, searchExecutionContext.getFullyQualifiedIndex().getName()); + assertEquals(expectedIndexName, searchShardTarget.getFullyQualifiedIndexName()); + assertEquals(clusterAlias, searchShardTarget.getClusterAlias()); + assertEquals(shardId, searchShardTarget.getShardId()); + + assertNull(searchContext.dfsResult()); + searchContext.addDfsResult(); + assertSame(searchShardTarget, searchContext.dfsResult().getSearchShardTarget()); + + assertNull(searchContext.queryResult()); + searchContext.addQueryResult(); + assertSame(searchShardTarget, searchContext.queryResult().getSearchShardTarget()); + + assertNull(searchContext.fetchResult()); + searchContext.addFetchResult(); + assertSame(searchShardTarget, searchContext.fetchResult().getSearchShardTarget()); + } + } + + /** + * While we have no NPE in DefaultContext constructor anymore, we still want to guard against it (or other failures) in the future to + * avoid leaking searchers. + */ + public void testCreateSearchContextFailure() throws Exception { + final String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); + final IndexService indexService = createIndex(index); + final SearchService service = getInstanceFromNode(SearchService.class); + final ShardId shardId = new ShardId(indexService.index(), 0); + final ShardSearchRequest request = new ShardSearchRequest(shardId, 0, null) { + @Override + public SearchType searchType() { + // induce an artificial NPE + throw new NullPointerException("expected"); + } + }; + try (ReaderContext reader = createReaderContext(indexService, indexService.getShard(shardId.id()))) { + NullPointerException e = expectThrows( + NullPointerException.class, + () -> service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) + ); + assertEquals("expected", e.getMessage()); + } + // Needs to busily assert because Engine#refreshNeeded can increase the refCount. + assertBusy( + () -> assertEquals("should have 2 store refs (IndexService + InternalEngine)", 2, indexService.getShard(0).store().refCount()) + ); + } + + public void testMatchNoDocsEmptyResponse() throws InterruptedException { + createIndex("index"); + Thread currentThread = Thread.currentThread(); + SearchService service = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().aggregation(AggregationBuilders.count("count").field("value"))); + ShardSearchRequest shardRequest = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 5, + AliasFilter.EMPTY, + 1.0f, + 0, + null + ); + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); + + { + CountDownLatch latch = new CountDownLatch(1); + shardRequest.source().query(new MatchAllQueryBuilder()); + service.executeQueryPhase(shardRequest, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult result) { + try { + assertNotSame(Thread.currentThread(), currentThread); + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + assertThat(result, instanceOf(QuerySearchResult.class)); + assertFalse(result.queryResult().isNull()); + assertNotNull(result.queryResult().topDocs()); + assertNotNull(result.queryResult().aggregations()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception exc) { + try { + throw new AssertionError(exc); + } finally { + latch.countDown(); + } + } + }); + latch.await(); + } + + { + CountDownLatch latch = new CountDownLatch(1); + shardRequest.source().query(new MatchNoneQueryBuilder()); + service.executeQueryPhase(shardRequest, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult result) { + try { + assertNotSame(Thread.currentThread(), currentThread); + assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); + assertThat(result, instanceOf(QuerySearchResult.class)); + assertFalse(result.queryResult().isNull()); + assertNotNull(result.queryResult().topDocs()); + assertNotNull(result.queryResult().aggregations()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception exc) { + try { + throw new AssertionError(exc); + } finally { + latch.countDown(); + } + } + }); + latch.await(); + } + + { + CountDownLatch latch = new CountDownLatch(1); + shardRequest.canReturnNullResponseIfMatchNoDocs(true); + service.executeQueryPhase(shardRequest, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult result) { + try { + // make sure we don't use the search threadpool + assertSame(Thread.currentThread(), currentThread); + assertThat(result, instanceOf(QuerySearchResult.class)); + assertTrue(result.queryResult().isNull()); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception e) { + try { + throw new AssertionError(e); + } finally { + latch.countDown(); + } + } + }); + latch.await(); + } + } + + public void testDeleteIndexWhileSearch() throws Exception { + createIndex("test"); + int numDocs = randomIntBetween(1, 20); + for (int i = 0; i < numDocs; i++) { + prepareIndex("test").setSource("f", "v").get(); + } + indicesAdmin().prepareRefresh("test").get(); + AtomicBoolean stopped = new AtomicBoolean(false); + Thread[] searchers = new Thread[randomIntBetween(1, 4)]; + CountDownLatch latch = new CountDownLatch(searchers.length); + for (int i = 0; i < searchers.length; i++) { + searchers[i] = new Thread(() -> { + latch.countDown(); + while (stopped.get() == false) { + try { + client().prepareSearch("test").setRequestCache(false).get().decRef(); + } catch (Exception ignored) { + return; + } + } + }); + searchers[i].start(); + } + latch.await(); + indicesAdmin().prepareDelete("test").get(); + stopped.set(true); + for (Thread searcher : searchers) { + searcher.join(); + } + } + + public void testLookUpSearchContext() throws Exception { + createIndex("index"); + SearchService searchService = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + List contextIds = new ArrayList<>(); + int numContexts = randomIntBetween(1, 10); + CountDownLatch latch = new CountDownLatch(1); + indexShard.getThreadPool().executor(ThreadPool.Names.SEARCH).execute(() -> { + try { + for (int i = 0; i < numContexts; i++) { + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + new SearchRequest().allowPartialSearchResults(true), + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + final ReaderContext context = searchService.createAndPutReaderContext( + request, + indexService, + indexShard, + indexShard.acquireSearcherSupplier(), + SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() + ); + assertThat(context.id().getId(), equalTo((long) (i + 1))); + contextIds.add(context.id()); + } + assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); + while (contextIds.isEmpty() == false) { + final ShardSearchContextId contextId = randomFrom(contextIds); + assertFalse(searchService.freeReaderContext(new ShardSearchContextId(UUIDs.randomBase64UUID(), contextId.getId()))); + assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); + if (randomBoolean()) { + assertTrue(searchService.freeReaderContext(contextId)); + } else { + assertTrue( + searchService.freeReaderContext((new ShardSearchContextId(contextId.getSessionId(), contextId.getId()))) + ); + } + contextIds.remove(contextId); + assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); + assertFalse(searchService.freeReaderContext(contextId)); + assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); + } + } finally { + latch.countDown(); + } + }); + latch.await(); + } + + public void testOpenReaderContext() { + createIndex("index"); + SearchService searchService = getInstanceFromNode(SearchService.class); + PlainActionFuture future = new PlainActionFuture<>(); + searchService.openReaderContext(new ShardId(resolveIndex("index"), 0), TimeValue.timeValueMinutes(between(1, 10)), future); + future.actionGet(); + assertThat(searchService.getActiveContexts(), equalTo(1)); + assertTrue(searchService.freeReaderContext(future.actionGet())); + } + + public void testCancelQueryPhaseEarly() throws Exception { + createIndex("index"); + final MockSearchService service = (MockSearchService) getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + + CountDownLatch latch1 = new CountDownLatch(1); + SearchShardTask task = new SearchShardTask(1, "", "", "", TaskId.EMPTY_TASK_ID, emptyMap()); + service.executeQueryPhase(request, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + service.freeReaderContext(searchPhaseResult.getContextId()); + latch1.countDown(); + } + + @Override + public void onFailure(Exception e) { + try { + fail("Search should not be cancelled"); + } finally { + latch1.countDown(); + } + } + }); + latch1.await(); + + CountDownLatch latch2 = new CountDownLatch(1); + service.executeDfsPhase(request, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + service.freeReaderContext(searchPhaseResult.getContextId()); + latch2.countDown(); + } + + @Override + public void onFailure(Exception e) { + try { + fail("Search should not be cancelled"); + } finally { + latch2.countDown(); + } + } + }); + latch2.await(); + + AtomicBoolean searchContextCreated = new AtomicBoolean(false); + service.setOnCreateSearchContext(c -> searchContextCreated.set(true)); + CountDownLatch latch3 = new CountDownLatch(1); + TaskCancelHelper.cancel(task, "simulated"); + service.executeQueryPhase(request, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + try { + fail("Search not cancelled early"); + } finally { + service.freeReaderContext(searchPhaseResult.getContextId()); + searchPhaseResult.decRef(); + latch3.countDown(); + } + } + + @Override + public void onFailure(Exception e) { + assertThat(e, is(instanceOf(TaskCancelledException.class))); + assertThat(e.getMessage(), is("task cancelled [simulated]")); + assertThat(((TaskCancelledException) e).status(), is(RestStatus.BAD_REQUEST)); + assertThat(searchContextCreated.get(), is(false)); + latch3.countDown(); + } + }); + latch3.await(); + + searchContextCreated.set(false); + CountDownLatch latch4 = new CountDownLatch(1); + service.executeDfsPhase(request, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + try { + fail("Search not cancelled early"); + } finally { + service.freeReaderContext(searchPhaseResult.getContextId()); + latch4.countDown(); + } + } + + @Override + public void onFailure(Exception e) { + assertThat(e, is(instanceOf(TaskCancelledException.class))); + assertThat(e.getMessage(), is("task cancelled [simulated]")); + assertThat(((TaskCancelledException) e).status(), is(RestStatus.BAD_REQUEST)); + assertThat(searchContextCreated.get(), is(false)); + latch4.countDown(); + } + }); + latch4.await(); + } + + public void testCancelFetchPhaseEarly() throws Exception { + createIndex("index"); + final MockSearchService service = (MockSearchService) getInstanceFromNode(SearchService.class); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + + AtomicBoolean searchContextCreated = new AtomicBoolean(false); + service.setOnCreateSearchContext(c -> searchContextCreated.set(true)); + + // Test fetch phase is cancelled early + String scrollId; + var searchResponse = client().search(searchRequest.allowPartialSearchResults(false).scroll(TimeValue.timeValueMinutes(10))).get(); + try { + scrollId = searchResponse.getScrollId(); + } finally { + searchResponse.decRef(); + } + + client().searchScroll(new SearchScrollRequest(scrollId)).get().decRef(); + assertThat(searchContextCreated.get(), is(true)); + + ClearScrollRequest clearScrollRequest = new ClearScrollRequest(); + clearScrollRequest.addScrollId(scrollId); + client().clearScroll(clearScrollRequest); + + searchResponse = client().search(searchRequest.allowPartialSearchResults(false).scroll(TimeValue.timeValueMinutes(10))).get(); + try { + scrollId = searchResponse.getScrollId(); + } finally { + searchResponse.decRef(); + } + searchContextCreated.set(false); + service.setOnCheckCancelled(t -> { + SearchShardTask task = new SearchShardTask(randomLong(), "transport", "action", "", TaskId.EMPTY_TASK_ID, emptyMap()); + TaskCancelHelper.cancel(task, "simulated"); + return task; + }); + CountDownLatch latch = new CountDownLatch(1); + client().searchScroll(new SearchScrollRequest(scrollId), new ActionListener<>() { + @Override + public void onResponse(SearchResponse searchResponse) { + try { + fail("Search not cancelled early"); + } finally { + latch.countDown(); + } + } + + @Override + public void onFailure(Exception e) { + Throwable cancelledExc = e.getCause().getCause(); + assertThat(cancelledExc, is(instanceOf(TaskCancelledException.class))); + assertThat(cancelledExc.getMessage(), is("task cancelled [simulated]")); + assertThat(((TaskCancelledException) cancelledExc).status(), is(RestStatus.BAD_REQUEST)); + latch.countDown(); + } + }); + latch.await(); + assertThat(searchContextCreated.get(), is(false)); + + clearScrollRequest.setScrollIds(singletonList(scrollId)); + client().clearScroll(clearScrollRequest); + } + + public void testWaitOnRefresh() throws ExecutionException, InterruptedException { + createIndex("index"); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueSeconds(30)); + searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); + + final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); + assertEquals(RestStatus.CREATED, response.status()); + + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null, + null, + null + ); + PlainActionFuture future = new PlainActionFuture<>(); + service.executeQueryPhase(request, task, future.delegateFailure((l, r) -> { + assertEquals(1, r.queryResult().getTotalHits().value); + l.onResponse(null); + })); + future.get(); + } + + public void testWaitOnRefreshFailsWithRefreshesDisabled() { + createIndex("index", Settings.builder().put("index.refresh_interval", "-1").build()); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueSeconds(30)); + searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); + + final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); + assertEquals(RestStatus.CREATED, response.status()); + + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); + PlainActionFuture future = new PlainActionFuture<>(); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null, + null, + null + ); + service.executeQueryPhase(request, task, future); + IllegalArgumentException illegalArgumentException = expectThrows(IllegalArgumentException.class, future::actionGet); + assertThat( + illegalArgumentException.getMessage(), + containsString("Cannot use wait_for_checkpoints with [index.refresh_interval=-1]") + ); + } + + public void testWaitOnRefreshFailsIfCheckpointNotIndexed() { + createIndex("index"); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + // Increased timeout to avoid cancelling the search task prior to its completion, + // as we expect to raise an Exception. Timeout itself is tested on the following `testWaitOnRefreshTimeout` test. + searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueMillis(randomIntBetween(200, 300))); + searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 1 })); + + final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); + assertEquals(RestStatus.CREATED, response.status()); + + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); + PlainActionFuture future = new PlainActionFuture<>(); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null, + null, + null + ); + service.executeQueryPhase(request, task, future); + + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, future::actionGet); + assertThat( + ex.getMessage(), + containsString("Cannot wait for unissued seqNo checkpoint [wait_for_checkpoint=1, max_issued_seqNo=0]") + ); + } + + public void testWaitOnRefreshTimeout() { + createIndex("index", Settings.builder().put("index.refresh_interval", "60s").build()); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueMillis(randomIntBetween(10, 100))); + searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); + + final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); + assertEquals(RestStatus.CREATED, response.status()); + + SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); + PlainActionFuture future = new PlainActionFuture<>(); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null, + null, + null + ); + service.executeQueryPhase(request, task, future); + + SearchTimeoutException ex = expectThrows(SearchTimeoutException.class, future::actionGet); + assertThat(ex.getMessage(), containsString("Wait for seq_no [0] refreshed timed out [")); + } + + public void testMinimalSearchSourceInShardRequests() { + createIndex("test"); + int numDocs = between(0, 10); + for (int i = 0; i < numDocs; i++) { + prepareIndex("test").setSource("id", Integer.toString(i)).get(); + } + indicesAdmin().prepareRefresh("test").get(); + + BytesReference pitId = client().execute( + TransportOpenPointInTimeAction.TYPE, + new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(10)) + ).actionGet().getPointInTimeId(); + final MockSearchService searchService = (MockSearchService) getInstanceFromNode(SearchService.class); + final List shardRequests = new CopyOnWriteArrayList<>(); + searchService.setOnCreateSearchContext(ctx -> shardRequests.add(ctx.request())); + try { + assertHitCount( + client().prepareSearch() + .setSource( + new SearchSourceBuilder().size(between(numDocs, numDocs * 2)).pointInTimeBuilder(new PointInTimeBuilder(pitId)) + ), + numDocs + ); + } finally { + client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); + } + assertThat(shardRequests, not(emptyList())); + for (ShardSearchRequest shardRequest : shardRequests) { + assertNotNull(shardRequest.source()); + assertNotNull(shardRequest.source().pointInTimeBuilder()); + assertThat(shardRequest.source().pointInTimeBuilder().getEncodedId(), equalTo(BytesArray.EMPTY)); + } + } + + public void testDfsQueryPhaseRewrite() { + createIndex("index"); + prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + final SearchService service = getInstanceFromNode(SearchService.class); + final IndicesService indicesService = getInstanceFromNode(IndicesService.class); + final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + final IndexShard indexShard = indexService.getShard(0); + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); + searchRequest.source(SearchSourceBuilder.searchSource().query(new TestRewriteCounterQueryBuilder())); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + indexShard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1.0f, + -1, + null + ); + final Engine.SearcherSupplier reader = indexShard.acquireSearcherSupplier(); + ReaderContext context = service.createAndPutReaderContext( + request, + indexService, + indexShard, + reader, + SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() + ); + PlainActionFuture plainActionFuture = new PlainActionFuture<>(); + service.executeQueryPhase( + new QuerySearchRequest(null, context.id(), request, new AggregatedDfs(Map.of(), Map.of(), 10)), + new SearchShardTask(42L, "", "", "", null, emptyMap()), + plainActionFuture + ); + + plainActionFuture.actionGet(); + assertThat(((TestRewriteCounterQueryBuilder) request.source().query()).asyncRewriteCount, equalTo(1)); + final ShardSearchContextId contextId = context.id(); + assertTrue(service.freeReaderContext(contextId)); + } + + public void testEnableSearchWorkerThreads() throws IOException { + IndexService indexService = createIndex("index", Settings.EMPTY); + IndexShard indexShard = indexService.getShard(0); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + new SearchRequest().allowPartialSearchResults(randomBoolean()), + indexShard.shardId(), + 0, + indexService.numberOfShards(), + AliasFilter.EMPTY, + 1f, + System.currentTimeMillis(), + null + ); + try (ReaderContext readerContext = createReaderContext(indexService, indexShard)) { + SearchService service = getInstanceFromNode(SearchService.class); + SearchShardTask task = new SearchShardTask(0, "type", "action", "description", null, emptyMap()); + + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, randomBoolean())) { + assertNotNull(searchContext.searcher().getExecutor()); + } + + try { + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) + .setPersistentSettings(Settings.builder().put(SEARCH_WORKER_THREADS_ENABLED.getKey(), false).build()) + .get(); + assertTrue(response.isAcknowledged()); + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, randomBoolean())) { + assertNull(searchContext.searcher().getExecutor()); + } + } finally { + // reset original default setting + client().admin() + .cluster() + .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) + .setPersistentSettings(Settings.builder().putNull(SEARCH_WORKER_THREADS_ENABLED.getKey()).build()) + .get(); + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, randomBoolean())) { + assertNotNull(searchContext.searcher().getExecutor()); + } + } + } + } + + /** + * Verify that a single slice is created for requests that don't support parallel collection, while an executor is still + * provided to the searcher to parallelize other operations. Also ensure multiple slices are created for requests that do support + * parallel collection. + */ + public void testSlicingBehaviourForParallelCollection() throws Exception { + IndexService indexService = createIndex("index", Settings.EMPTY); + ThreadPoolExecutor executor = (ThreadPoolExecutor) indexService.getThreadPool().executor(ThreadPool.Names.SEARCH); + final int configuredMaxPoolSize = 10; + executor.setMaximumPoolSize(configuredMaxPoolSize); // We set this explicitly to be independent of CPU cores. + int numDocs = randomIntBetween(50, 100); + for (int i = 0; i < numDocs; i++) { + prepareIndex("index").setId(String.valueOf(i)).setSource("field", "value").get(); + if (i % 5 == 0) { + indicesAdmin().prepareRefresh("index").get(); + } + } + final IndexShard indexShard = indexService.getShard(0); + ShardSearchRequest request = new ShardSearchRequest( + OriginalIndices.NONE, + new SearchRequest().allowPartialSearchResults(randomBoolean()), + indexShard.shardId(), + 0, + indexService.numberOfShards(), + AliasFilter.EMPTY, + 1f, + System.currentTimeMillis(), + null + ); + SearchService service = getInstanceFromNode(SearchService.class); + NonCountingTermQuery termQuery = new NonCountingTermQuery(new Term("field", "value")); + assertEquals(0, executor.getCompletedTaskCount()); + try (ReaderContext readerContext = createReaderContext(indexService, indexShard)) { + SearchShardTask task = new SearchShardTask(0, "type", "action", "description", null, emptyMap()); + { + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, true)) { + ContextIndexSearcher searcher = searchContext.searcher(); + assertNotNull(searcher.getExecutor()); + + final int maxPoolSize = executor.getMaximumPoolSize(); + assertEquals( + "Sanity check to ensure this isn't the default of 1 when pool size is unset", + configuredMaxPoolSize, + maxPoolSize + ); + + final int expectedSlices = ContextIndexSearcher.computeSlices( + searcher.getIndexReader().leaves(), + maxPoolSize, + 1 + ).length; + assertNotEquals("Sanity check to ensure this isn't the default of 1 when pool size is unset", 1, expectedSlices); + + final long priorExecutorTaskCount = executor.getCompletedTaskCount(); + searcher.search(termQuery, new TotalHitCountCollectorManager()); + assertBusy( + () -> assertEquals( + "DFS supports parallel collection, so the number of slices should be > 1.", + expectedSlices - 1, // one slice executes on the calling thread + executor.getCompletedTaskCount() - priorExecutorTaskCount + ) + ); + } + } + { + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.QUERY, true)) { + ContextIndexSearcher searcher = searchContext.searcher(); + assertNotNull(searcher.getExecutor()); + + final int maxPoolSize = executor.getMaximumPoolSize(); + assertEquals( + "Sanity check to ensure this isn't the default of 1 when pool size is unset", + configuredMaxPoolSize, + maxPoolSize + ); + + final int expectedSlices = ContextIndexSearcher.computeSlices( + searcher.getIndexReader().leaves(), + maxPoolSize, + 1 + ).length; + assertNotEquals("Sanity check to ensure this isn't the default of 1 when pool size is unset", 1, expectedSlices); + + final long priorExecutorTaskCount = executor.getCompletedTaskCount(); + searcher.search(termQuery, new TotalHitCountCollectorManager()); + assertBusy( + () -> assertEquals( + "QUERY supports parallel collection when enabled, so the number of slices should be > 1.", + expectedSlices - 1, // one slice executes on the calling thread + executor.getCompletedTaskCount() - priorExecutorTaskCount + ) + ); + } + } + { + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.FETCH, true)) { + ContextIndexSearcher searcher = searchContext.searcher(); + assertNull(searcher.getExecutor()); + final long priorExecutorTaskCount = executor.getCompletedTaskCount(); + searcher.search(termQuery, new TotalHitCountCollectorManager()); + assertBusy( + () -> assertEquals( + "The number of slices should be 1 as FETCH does not support parallel collection and thus runs on the calling" + + " thread.", + 0, + executor.getCompletedTaskCount() - priorExecutorTaskCount + ) + ); + } + } + { + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.NONE, true)) { + ContextIndexSearcher searcher = searchContext.searcher(); + assertNull(searcher.getExecutor()); + final long priorExecutorTaskCount = executor.getCompletedTaskCount(); + searcher.search(termQuery, new TotalHitCountCollectorManager()); + assertBusy( + () -> assertEquals( + "The number of slices should be 1 as NONE does not support parallel collection.", + 0, // zero since one slice executes on the calling thread + executor.getCompletedTaskCount() - priorExecutorTaskCount + ) + ); + } + } + + try { + ClusterUpdateSettingsResponse response = client().admin() + .cluster() + .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) + .setPersistentSettings(Settings.builder().put(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED.getKey(), false).build()) + .get(); + assertTrue(response.isAcknowledged()); + { + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.QUERY, true)) { + ContextIndexSearcher searcher = searchContext.searcher(); + assertNull(searcher.getExecutor()); + final long priorExecutorTaskCount = executor.getCompletedTaskCount(); + searcher.search(termQuery, new TotalHitCountCollectorManager()); + assertBusy( + () -> assertEquals( + "The number of slices should be 1 when QUERY parallel collection is disabled.", + 0, // zero since one slice executes on the calling thread + executor.getCompletedTaskCount() - priorExecutorTaskCount + ) + ); + } + } + } finally { + // Reset to the original default setting and check to ensure it takes effect. + client().admin() + .cluster() + .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) + .setPersistentSettings(Settings.builder().putNull(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED.getKey()).build()) + .get(); + { + try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.QUERY, true)) { + ContextIndexSearcher searcher = searchContext.searcher(); + assertNotNull(searcher.getExecutor()); + + final int maxPoolSize = executor.getMaximumPoolSize(); + assertEquals( + "Sanity check to ensure this isn't the default of 1 when pool size is unset", + configuredMaxPoolSize, + maxPoolSize + ); + + final int expectedSlices = ContextIndexSearcher.computeSlices( + searcher.getIndexReader().leaves(), + maxPoolSize, + 1 + ).length; + assertNotEquals("Sanity check to ensure this isn't the default of 1 when pool size is unset", 1, expectedSlices); + + final long priorExecutorTaskCount = executor.getCompletedTaskCount(); + searcher.search(termQuery, new TotalHitCountCollectorManager()); + assertBusy( + () -> assertEquals( + "QUERY supports parallel collection when enabled, so the number of slices should be > 1.", + expectedSlices - 1, // one slice executes on the calling thread + executor.getCompletedTaskCount() - priorExecutorTaskCount + ) + ); + } + } + } + } + } + + private static ReaderContext createReaderContext(IndexService indexService, IndexShard indexShard) { + return new ReaderContext( + new ShardSearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong()), + indexService, + indexShard, + indexShard.acquireSearcherSupplier(), + randomNonNegativeLong(), + false + ); + } + + private static class TestRewriteCounterQueryBuilder extends AbstractQueryBuilder { + + final int asyncRewriteCount; + final Supplier fetched; + + TestRewriteCounterQueryBuilder() { + asyncRewriteCount = 0; + fetched = null; + } + + private TestRewriteCounterQueryBuilder(int asyncRewriteCount, Supplier fetched) { + this.asyncRewriteCount = asyncRewriteCount; + this.fetched = fetched; + } + + @Override + public String getWriteableName() { + return "test_query"; + } + + @Override + public TransportVersion getMinimalSupportedVersion() { + return TransportVersions.ZERO; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException {} + + @Override + protected void doXContent(XContentBuilder builder, Params params) throws IOException {} + + @Override + protected Query doToQuery(SearchExecutionContext context) throws IOException { + return new MatchAllDocsQuery(); + } + + @Override + protected boolean doEquals(TestRewriteCounterQueryBuilder other) { + return true; + } + + @Override + protected int doHashCode() { + return 42; + } + + @Override + protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws IOException { + if (asyncRewriteCount > 0) { + return this; + } + if (fetched != null) { + if (fetched.get() == null) { + return this; + } + assert fetched.get(); + return new TestRewriteCounterQueryBuilder(1, null); + } + if (queryRewriteContext.convertToDataRewriteContext() != null) { + SetOnce awaitingFetch = new SetOnce<>(); + queryRewriteContext.registerAsyncAction((c, l) -> { + awaitingFetch.set(true); + l.onResponse(null); + }); + return new TestRewriteCounterQueryBuilder(0, awaitingFetch::get); + } + return this; + } + } +} diff --git a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java index 642804730a144..31bcab31ca8a7 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -6,3121 +6,298 @@ * your election, the "Elastic License 2.0", the "GNU Affero General Public * License v3.0 only", or the "Server Side Public License, v 1". */ + package org.elasticsearch.search; -import org.apache.lucene.index.DirectoryReader; -import org.apache.lucene.index.FilterDirectoryReader; -import org.apache.lucene.index.LeafReader; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.MatchAllDocsQuery; -import org.apache.lucene.search.Query; -import org.apache.lucene.search.ScoreDoc; -import org.apache.lucene.search.TopDocs; -import org.apache.lucene.search.TotalHitCountCollectorManager; -import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.util.SetOnce; -import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.ElasticsearchTimeoutException; -import org.elasticsearch.TransportVersion; -import org.elasticsearch.TransportVersions; -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.DocWriteResponse; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.SortField; +import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.OriginalIndices; -import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; -import org.elasticsearch.action.search.ClearScrollRequest; -import org.elasticsearch.action.search.ClosePointInTimeRequest; -import org.elasticsearch.action.search.OpenPointInTimeRequest; -import org.elasticsearch.action.search.SearchPhaseController; -import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; -import org.elasticsearch.action.search.SearchResponse; -import org.elasticsearch.action.search.SearchScrollRequest; -import org.elasticsearch.action.search.SearchShardTask; -import org.elasticsearch.action.search.SearchType; -import org.elasticsearch.action.search.TransportClosePointInTimeAction; -import org.elasticsearch.action.search.TransportOpenPointInTimeAction; -import org.elasticsearch.action.search.TransportSearchAction; -import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.PlainActionFuture; -import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.IndexMetadata; -import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.cluster.routing.TestShardRouting; -import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.common.breaker.CircuitBreakingException; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; -import org.elasticsearch.common.bytes.BytesArray; -import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.core.AbstractRefCounted; -import org.elasticsearch.core.TimeValue; -import org.elasticsearch.index.Index; -import org.elasticsearch.index.IndexModule; -import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.IndexVersion; import org.elasticsearch.index.engine.Engine; -import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.fielddata.FieldDataContext; +import org.elasticsearch.index.fielddata.IndexFieldData; +import org.elasticsearch.index.fielddata.LeafFieldData; +import org.elasticsearch.index.mapper.KeywordFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperBuilderContext; +import org.elasticsearch.index.mapper.MapperMetrics; +import org.elasticsearch.index.mapper.Mapping; +import org.elasticsearch.index.mapper.MappingLookup; +import org.elasticsearch.index.mapper.MetadataFieldMapper; +import org.elasticsearch.index.mapper.ObjectMapper; +import org.elasticsearch.index.mapper.RootObjectMapper; import org.elasticsearch.index.query.MatchAllQueryBuilder; import org.elasticsearch.index.query.MatchNoneQueryBuilder; -import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryRewriteContext; import org.elasticsearch.index.query.SearchExecutionContext; -import org.elasticsearch.index.query.TermQueryBuilder; -import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.shard.IndexShard; -import org.elasticsearch.index.shard.SearchOperationListener; +import org.elasticsearch.index.shard.IndexShardTestCase; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.indices.settings.InternalOrPrivateSettingsPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.plugins.SearchPlugin; -import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.script.MockScriptEngine; -import org.elasticsearch.script.MockScriptPlugin; -import org.elasticsearch.script.Script; -import org.elasticsearch.script.ScriptType; -import org.elasticsearch.search.SearchService.ResultsType; -import org.elasticsearch.search.aggregations.AggregationBuilders; -import org.elasticsearch.search.aggregations.AggregationReduceContext; -import org.elasticsearch.search.aggregations.MultiBucketConsumerService; -import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregationBuilder; -import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; -import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; -import org.elasticsearch.search.aggregations.support.AggregationContext; -import org.elasticsearch.search.aggregations.support.ValueType; -import org.elasticsearch.search.builder.PointInTimeBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.collapse.CollapseBuilder; -import org.elasticsearch.search.dfs.AggregatedDfs; -import org.elasticsearch.search.fetch.FetchSearchResult; -import org.elasticsearch.search.fetch.ShardFetchRequest; -import org.elasticsearch.search.fetch.ShardFetchSearchRequest; -import org.elasticsearch.search.fetch.subphase.FieldAndFormat; import org.elasticsearch.search.internal.AliasFilter; -import org.elasticsearch.search.internal.ContextIndexSearcher; -import org.elasticsearch.search.internal.ReaderContext; -import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; -import org.elasticsearch.search.query.NonCountingTermQuery; -import org.elasticsearch.search.query.QuerySearchRequest; -import org.elasticsearch.search.query.QuerySearchResult; -import org.elasticsearch.search.rank.RankBuilder; -import org.elasticsearch.search.rank.RankDoc; -import org.elasticsearch.search.rank.RankShardResult; -import org.elasticsearch.search.rank.TestRankBuilder; -import org.elasticsearch.search.rank.TestRankShardResult; -import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; -import org.elasticsearch.search.rank.context.QueryPhaseRankShardContext; -import org.elasticsearch.search.rank.context.RankFeaturePhaseRankCoordinatorContext; -import org.elasticsearch.search.rank.context.RankFeaturePhaseRankShardContext; -import org.elasticsearch.search.rank.feature.RankFeatureDoc; -import org.elasticsearch.search.rank.feature.RankFeatureResult; -import org.elasticsearch.search.rank.feature.RankFeatureShardRequest; -import org.elasticsearch.search.rank.feature.RankFeatureShardResult; -import org.elasticsearch.search.slice.SliceBuilder; -import org.elasticsearch.search.suggest.SuggestBuilder; -import org.elasticsearch.tasks.TaskCancelHelper; -import org.elasticsearch.tasks.TaskCancelledException; -import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.test.ESSingleNodeTestCase; -import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; -import org.elasticsearch.threadpool.ThreadPool; -import org.elasticsearch.xcontent.XContentBuilder; -import org.elasticsearch.xcontent.json.JsonXContent; -import org.junit.Before; +import org.elasticsearch.search.sort.BucketedSort; +import org.elasticsearch.search.sort.MinAndMax; +import org.elasticsearch.search.sort.SortOrder; +import org.elasticsearch.xcontent.XContentParserConfiguration; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.Comparator; -import java.util.LinkedList; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import java.util.concurrent.CopyOnWriteArrayList; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Semaphore; -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Consumer; -import java.util.function.Function; -import java.util.function.IntConsumer; -import java.util.function.Supplier; - -import static java.util.Collections.emptyList; -import static java.util.Collections.emptyMap; -import static java.util.Collections.singletonList; -import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; -import static org.elasticsearch.indices.cluster.IndicesClusterStateService.AllocatedIndices.IndexRemovalReason.DELETED; -import static org.elasticsearch.search.SearchService.DEFAULT_SIZE; -import static org.elasticsearch.search.SearchService.QUERY_PHASE_PARALLEL_COLLECTION_ENABLED; -import static org.elasticsearch.search.SearchService.SEARCH_WORKER_THREADS_ENABLED; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; -import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; -import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.CoreMatchers.is; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.CoreMatchers.startsWith; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.not; -import static org.mockito.Mockito.mock; +import java.util.function.BiFunction; +import java.util.function.Predicate; -public class SearchServiceTests extends ESSingleNodeTestCase { +public class SearchServiceTests extends IndexShardTestCase { - @Override - protected boolean resetNodeAfterTest() { - return true; + public void testCanMatchMatchAll() throws IOException { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().query(new MatchAllQueryBuilder())); + doTestCanMatch(searchRequest, null, true, null, false); } - @Override - protected Collection> getPlugins() { - return pluginList( - FailOnRewriteQueryPlugin.class, - CustomScriptPlugin.class, - ReaderWrapperCountPlugin.class, - InternalOrPrivateSettingsPlugin.class, - MockSearchService.TestPlugin.class - ); + public void testCanMatchMatchNone() throws IOException { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().query(new MatchNoneQueryBuilder())); + doTestCanMatch(searchRequest, null, false, null, false); } - public static class ReaderWrapperCountPlugin extends Plugin { - @Override - public void onIndexModule(IndexModule indexModule) { - indexModule.setReaderWrapper(service -> SearchServiceTests::apply); - } + public void testCanMatchMatchNoneWithException() throws IOException { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().query(new MatchNoneQueryBuilder())); + doTestCanMatch(searchRequest, null, true, null, true); } - @Before - public void resetCount() { - numWrapInvocations = new AtomicInteger(0); + public void testCanMatchKeywordSortedQueryMatchNone() throws IOException { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().sort("field").query(new MatchNoneQueryBuilder())); + SortField sortField = new SortField("field", SortField.Type.STRING); + doTestCanMatch(searchRequest, sortField, false, null, false); } - private static AtomicInteger numWrapInvocations = new AtomicInteger(0); - - private static DirectoryReader apply(DirectoryReader directoryReader) throws IOException { - numWrapInvocations.incrementAndGet(); - return new FilterDirectoryReader(directoryReader, new FilterDirectoryReader.SubReaderWrapper() { - @Override - public LeafReader wrap(LeafReader reader) { - return reader; - } - }) { - @Override - protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException { - return in; - } + public void testCanMatchKeywordSortedQueryMatchAll() throws IOException { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().sort("field").query(new MatchAllQueryBuilder())); + SortField sortField = new SortField("field", SortField.Type.STRING); + MinAndMax expectedMinAndMax = new MinAndMax<>(new BytesRef("value"), new BytesRef("value")); + doTestCanMatch(searchRequest, sortField, true, expectedMinAndMax, false); + } + public void testCanMatchKeywordSortedQueryMatchNoneWithException() throws IOException { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().sort("field").query(new MatchNoneQueryBuilder())); + // provide a sort field that throws exception + SortField sortField = new SortField("field", SortField.Type.STRING) { @Override - public CacheHelper getReaderCacheHelper() { - return directoryReader.getReaderCacheHelper(); + public Type getType() { + throw new UnsupportedOperationException(); } }; + doTestCanMatch(searchRequest, sortField, false, null, false); } - public static class CustomScriptPlugin extends MockScriptPlugin { - - static final String DUMMY_SCRIPT = "dummyScript"; - - @Override - protected Map, Object>> pluginScripts() { - return Collections.singletonMap(DUMMY_SCRIPT, vars -> "dummy"); - } - - @Override - public void onIndexModule(IndexModule indexModule) { - indexModule.addSearchOperationListener(new SearchOperationListener() { - @Override - public void onFetchPhase(SearchContext context, long tookInNanos) { - if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) { - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search_throttled]")); - } else { - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); - } - } - - @Override - public void onQueryPhase(SearchContext context, long tookInNanos) { - if ("throttled_threadpool_index".equals(context.indexShard().shardId().getIndex().getName())) { - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search_throttled]")); - } else { - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); - } - } - }); - } - } - - @Override - protected Settings nodeSettings() { - return Settings.builder().put("search.default_search_timeout", "5s").build(); - } - - public void testClearOnClose() { - createIndex("index"); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertResponse( - client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), - searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) - ); - SearchService service = getInstanceFromNode(SearchService.class); - - assertEquals(1, service.getActiveContexts()); - service.doClose(); // this kills the keep-alive reaper we have to reset the node after this test - assertEquals(0, service.getActiveContexts()); - } - - public void testClearOnStop() { - createIndex("index"); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertResponse( - client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), - searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) - ); - SearchService service = getInstanceFromNode(SearchService.class); - - assertEquals(1, service.getActiveContexts()); - service.doStop(); - assertEquals(0, service.getActiveContexts()); - } - - public void testClearIndexDelete() { - createIndex("index"); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertResponse( - client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), - searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) - ); - SearchService service = getInstanceFromNode(SearchService.class); - - assertEquals(1, service.getActiveContexts()); - assertAcked(indicesAdmin().prepareDelete("index")); - awaitIndexShardCloseAsyncTasks(); - assertEquals(0, service.getActiveContexts()); - } - - public void testCloseSearchContextOnRewriteException() { - // if refresh happens while checking the exception, the subsequent reference count might not match, so we switch it off - createIndex("index", Settings.builder().put("index.refresh_interval", -1).build()); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - - SearchService service = getInstanceFromNode(SearchService.class); - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - IndexShard indexShard = indexService.getShard(0); - - final int activeContexts = service.getActiveContexts(); - final int activeRefs = indexShard.store().refCount(); - expectThrows( - SearchPhaseExecutionException.class, - () -> client().prepareSearch("index").setQuery(new FailOnRewriteQueryBuilder()).get() - ); - assertEquals(activeContexts, service.getActiveContexts()); - assertEquals(activeRefs, indexShard.store().refCount()); - } - - public void testSearchWhileIndexDeleted() throws InterruptedException { - createIndex("index"); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - - SearchService service = getInstanceFromNode(SearchService.class); - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - IndexShard indexShard = indexService.getShard(0); - AtomicBoolean running = new AtomicBoolean(true); - CountDownLatch startGun = new CountDownLatch(1); - final int permitCount = 100; - Semaphore semaphore = new Semaphore(permitCount); - ShardRouting routing = TestShardRouting.newShardRouting( - indexShard.shardId(), - randomAlphaOfLength(5), - randomBoolean(), - ShardRoutingState.INITIALIZING - ); - final Thread thread = new Thread(() -> { - startGun.countDown(); - while (running.get()) { - if (randomBoolean()) { - service.afterIndexRemoved(indexService.index(), indexService.getIndexSettings(), DELETED); - } else { - service.beforeIndexShardCreated(routing, indexService.getIndexSettings().getSettings()); - } - if (randomBoolean()) { - // here we trigger some refreshes to ensure the IR go out of scope such that we hit ACE if we access a search - // context in a non-sane way. - try { - semaphore.acquire(); - } catch (InterruptedException e) { - throw new AssertionError(e); - } - prepareIndex("index").setSource("field", "value") - .setRefreshPolicy(randomFrom(WriteRequest.RefreshPolicy.values())) - .execute(ActionListener.running(semaphore::release)); - } - } - }); - thread.start(); - startGun.await(); - try { - final int rounds = scaledRandomIntBetween(100, 10000); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - SearchRequest scrollSearchRequest = new SearchRequest().allowPartialSearchResults(true) - .scroll(new Scroll(TimeValue.timeValueMinutes(1))); - for (int i = 0; i < rounds; i++) { - try { - try { - PlainActionFuture result = new PlainActionFuture<>(); - final boolean useScroll = randomBoolean(); - service.executeQueryPhase( - new ShardSearchRequest( - OriginalIndices.NONE, - useScroll ? scrollSearchRequest : searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ), - new SearchShardTask(123L, "", "", "", null, emptyMap()), - result.delegateFailure((l, r) -> { - r.incRef(); - l.onResponse(r); - }) - ); - final SearchPhaseResult searchPhaseResult = result.get(); - try { - List intCursors = new ArrayList<>(1); - intCursors.add(0); - ShardFetchRequest req = new ShardFetchRequest( - searchPhaseResult.getContextId(), - intCursors, - null/* not a scroll */ - ); - PlainActionFuture listener = new PlainActionFuture<>(); - service.executeFetchPhase(req, new SearchShardTask(123L, "", "", "", null, emptyMap()), listener); - listener.get(); - if (useScroll) { - // have to free context since this test does not remove the index from IndicesService. - service.freeReaderContext(searchPhaseResult.getContextId()); - } - } finally { - searchPhaseResult.decRef(); - } - } catch (ExecutionException ex) { - assertThat(ex.getCause(), instanceOf(RuntimeException.class)); - throw ((RuntimeException) ex.getCause()); - } - } catch (AlreadyClosedException ex) { - throw ex; - } catch (IllegalStateException ex) { - assertEquals(AbstractRefCounted.ALREADY_CLOSED_MESSAGE, ex.getMessage()); - } catch (SearchContextMissingException ex) { - // that's fine - } + public void testCanMatchKeywordSortedQueryMatchAllWithException() throws IOException { + SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) + .source(new SearchSourceBuilder().sort("field").query(new MatchAllQueryBuilder())); + // provide a sort field that throws exception + SortField sortField = new SortField("field", SortField.Type.STRING) { + @Override + public Type getType() { + throw new UnsupportedOperationException(); } - } finally { - running.set(false); - thread.join(); - semaphore.acquire(permitCount); - } - - assertEquals(0, service.getActiveContexts()); - - SearchStats.Stats totalStats = indexShard.searchStats().getTotal(); - assertEquals(0, totalStats.getQueryCurrent()); - assertEquals(0, totalStats.getScrollCurrent()); - assertEquals(0, totalStats.getFetchCurrent()); + }; + doTestCanMatch(searchRequest, sortField, true, null, false); } - public void testRankFeaturePhaseSearchPhases() throws InterruptedException, ExecutionException { - final String indexName = "index"; - final String rankFeatureFieldName = "field"; - final String searchFieldName = "search_field"; - final String searchFieldValue = "some_value"; - final String fetchFieldName = "fetch_field"; - final String fetchFieldValue = "fetch_value"; - - final int minDocs = 3; - final int maxDocs = 10; - int numDocs = between(minDocs, maxDocs); - createIndex(indexName); - // index some documents - for (int i = 0; i < numDocs; i++) { - prepareIndex(indexName).setId(String.valueOf(i)) - .setSource( - rankFeatureFieldName, - "aardvark_" + i, - searchFieldName, - searchFieldValue, - fetchFieldName, - fetchFieldValue + "_" + i - ) - .get(); - } - indicesAdmin().prepareRefresh(indexName).get(); - - final SearchService service = getInstanceFromNode(SearchService.class); - - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex(indexName)); - final IndexShard indexShard = indexService.getShard(0); - SearchShardTask searchTask = new SearchShardTask(123L, "", "", "", null, emptyMap()); - - // create a SearchRequest that will return all documents and defines a TestRankBuilder with shard-level only operations - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true) - .source( - new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) - .size(DEFAULT_SIZE) - .fetchField(fetchFieldName) - .rankBuilder( - // here we override only the shard-level contexts - new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - @Override - public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { - return new QueryPhaseRankShardContext(queries, from) { - - @Override - public int rankWindowSize() { - return DEFAULT_RANK_WINDOW_SIZE; - } - - @Override - public RankShardResult combineQueryPhaseResults(List rankResults) { - // we know we have just 1 query, so return all the docs from it - return new TestRankShardResult( - Arrays.stream(rankResults.get(0).scoreDocs) - .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) - .limit(rankWindowSize()) - .toArray(RankDoc[]::new) - ); - } - }; - } - - @Override - public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { - return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { - @Override - public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { - RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; - for (int i = 0; i < hits.getHits().length; i++) { - SearchHit hit = hits.getHits()[i]; - rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); - rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); - rankFeatureDocs[i].score = (numDocs - i) + randomFloat(); - rankFeatureDocs[i].rank = i + 1; - } - return new RankFeatureShardResult(rankFeatureDocs); - } - }; - } - } - ) - ); - - ShardSearchRequest request = new ShardSearchRequest( + private void doTestCanMatch( + SearchRequest searchRequest, + SortField sortField, + boolean expectedCanMatch, + MinAndMax expectedMinAndMax, + boolean throwException + ) throws IOException { + ShardSearchRequest shardRequest = new ShardSearchRequest( OriginalIndices.NONE, searchRequest, - indexShard.shardId(), + new ShardId("index", "index", 0), 0, - 1, + 5, AliasFilter.EMPTY, 1.0f, - -1, + 0, null ); - QuerySearchResult queryResult = null; - RankFeatureResult rankResult = null; + IndexFieldData indexFieldData = indexFieldData(sortField); + IndexShard indexShard = newShard(true); try { - // Execute the query phase and store the result in a SearchPhaseResult container using a PlainActionFuture - PlainActionFuture queryPhaseResults = new PlainActionFuture<>(); - service.executeQueryPhase(request, searchTask, queryPhaseResults); - queryResult = (QuerySearchResult) queryPhaseResults.get(); - - // these are the matched docs from the query phase - final RankDoc[] queryRankDocs = ((TestRankShardResult) queryResult.getRankShardResult()).testRankDocs; - - // assume that we have cut down to these from the coordinator node as the top-docs to run the rank feature phase upon - List topRankWindowSizeDocs = randomNonEmptySubsetOf(Arrays.stream(queryRankDocs).map(x -> x.doc).toList()); - - // now we create a RankFeatureShardRequest to extract feature info for the top-docs above - RankFeatureShardRequest rankFeatureShardRequest = new RankFeatureShardRequest( - OriginalIndices.NONE, - queryResult.getContextId(), // use the context from the query phase - request, - topRankWindowSizeDocs - ); - PlainActionFuture rankPhaseResults = new PlainActionFuture<>(); - service.executeRankFeaturePhase(rankFeatureShardRequest, searchTask, rankPhaseResults); - rankResult = rankPhaseResults.get(); - - assertNotNull(rankResult); - assertNotNull(rankResult.rankFeatureResult()); - RankFeatureShardResult rankFeatureShardResult = rankResult.rankFeatureResult().shardResult(); - assertNotNull(rankFeatureShardResult); - - List sortedRankWindowDocs = topRankWindowSizeDocs.stream().sorted().toList(); - assertEquals(sortedRankWindowDocs.size(), rankFeatureShardResult.rankFeatureDocs.length); - for (int i = 0; i < sortedRankWindowDocs.size(); i++) { - assertEquals((long) sortedRankWindowDocs.get(i), rankFeatureShardResult.rankFeatureDocs[i].doc); - assertEquals(rankFeatureShardResult.rankFeatureDocs[i].featureData, "aardvark_" + sortedRankWindowDocs.get(i)); - } - - List globalTopKResults = randomNonEmptySubsetOf( - Arrays.stream(rankFeatureShardResult.rankFeatureDocs).map(x -> x.doc).toList() - ); - - // finally let's create a fetch request to bring back fetch info for the top results - ShardFetchSearchRequest fetchRequest = new ShardFetchSearchRequest( - OriginalIndices.NONE, - rankResult.getContextId(), - request, - globalTopKResults, - null, - null, - rankResult.getRescoreDocIds(), - null - ); - - // execute fetch phase and perform any validations once we retrieve the response - // the difference in how we do assertions here is needed because once the transport service sends back the response - // it decrements the reference to the FetchSearchResult (through the ActionListener#respondAndRelease) and sets hits to null - PlainActionFuture fetchListener = new PlainActionFuture<>() { - @Override - public void onResponse(FetchSearchResult fetchSearchResult) { - assertNotNull(fetchSearchResult); - assertNotNull(fetchSearchResult.hits()); - - int totalHits = fetchSearchResult.hits().getHits().length; - assertEquals(globalTopKResults.size(), totalHits); - for (int i = 0; i < totalHits; i++) { - // rank and score are set by the SearchPhaseController#merge so no need to validate that here - SearchHit hit = fetchSearchResult.hits().getAt(i); - assertNotNull(hit.getFields().get(fetchFieldName)); - assertEquals(hit.getFields().get(fetchFieldName).getValue(), fetchFieldValue + "_" + hit.docId()); - } - super.onResponse(fetchSearchResult); - } - - @Override - public void onFailure(Exception e) { - super.onFailure(e); - throw new AssertionError("No failure should have been raised", e); - } - }; - service.executeFetchPhase(fetchRequest, searchTask, fetchListener); - fetchListener.get(); - } catch (Exception ex) { - if (queryResult != null) { - if (queryResult.hasReferences()) { - queryResult.decRef(); - } - service.freeReaderContext(queryResult.getContextId()); - } - if (rankResult != null && rankResult.hasReferences()) { - rankResult.decRef(); - } - throw ex; - } - } - - public void testRankFeaturePhaseUsingClient() { - final String indexName = "index"; - final String rankFeatureFieldName = "field"; - final String searchFieldName = "search_field"; - final String searchFieldValue = "some_value"; - final String fetchFieldName = "fetch_field"; - final String fetchFieldValue = "fetch_value"; - - final int minDocs = 4; - final int maxDocs = 10; - int numDocs = between(minDocs, maxDocs); - createIndex(indexName); - // index some documents - for (int i = 0; i < numDocs; i++) { - prepareIndex(indexName).setId(String.valueOf(i)) - .setSource( - rankFeatureFieldName, - "aardvark_" + i, - searchFieldName, - searchFieldValue, - fetchFieldName, - fetchFieldValue + "_" + i - ) - .get(); - } - indicesAdmin().prepareRefresh(indexName).get(); - - ElasticsearchAssertions.assertResponse( - client().prepareSearch(indexName) - .setSource( - new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) - .size(2) - .from(2) - .fetchField(fetchFieldName) - .rankBuilder( - // here we override only the shard-level contexts - new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - - // no need for more than one queries - @Override - public boolean isCompoundBuilder() { - return false; - } - - @Override - public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( - int size, - int from, - Client client - ) { - return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { - @Override - protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { - float[] scores = new float[featureDocs.length]; - for (int i = 0; i < featureDocs.length; i++) { - scores[i] = featureDocs[i].score; - } - scoreListener.onResponse(scores); - } - }; - } - - @Override - public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { - return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - @Override - public ScoreDoc[] rankQueryPhaseResults( - List querySearchResults, - SearchPhaseController.TopDocsStats topDocStats - ) { - List rankDocs = new ArrayList<>(); - for (int i = 0; i < querySearchResults.size(); i++) { - QuerySearchResult querySearchResult = querySearchResults.get(i); - TestRankShardResult shardResult = (TestRankShardResult) querySearchResult - .getRankShardResult(); - for (RankDoc trd : shardResult.testRankDocs) { - trd.shardIndex = i; - rankDocs.add(trd); - } - } - rankDocs.sort(Comparator.comparing((RankDoc doc) -> doc.score).reversed()); - RankDoc[] topResults = rankDocs.stream().limit(rankWindowSize).toArray(RankDoc[]::new); - topDocStats.fetchHits = topResults.length; - return topResults; - } - }; - } - - @Override - public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { - return new QueryPhaseRankShardContext(queries, from) { - - @Override - public int rankWindowSize() { - return DEFAULT_RANK_WINDOW_SIZE; - } - - @Override - public RankShardResult combineQueryPhaseResults(List rankResults) { - // we know we have just 1 query, so return all the docs from it - return new TestRankShardResult( - Arrays.stream(rankResults.get(0).scoreDocs) - .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) - .limit(rankWindowSize()) - .toArray(RankDoc[]::new) - ); - } - }; - } - - @Override - public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { - return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { - @Override - public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { - RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; - for (int i = 0; i < hits.getHits().length; i++) { - SearchHit hit = hits.getHits()[i]; - rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); - rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); - rankFeatureDocs[i].score = randomFloat(); - rankFeatureDocs[i].rank = i + 1; - } - return new RankFeatureShardResult(rankFeatureDocs); - } - }; - } - } - ) - ), - (response) -> { - SearchHits hits = response.getHits(); - assertEquals(hits.getTotalHits().value, numDocs); - assertEquals(hits.getHits().length, 2); - int index = 0; - for (SearchHit hit : hits.getHits()) { - assertEquals(hit.getRank(), 3 + index); - assertTrue(hit.getScore() >= 0); - assertEquals(hit.getFields().get(fetchFieldName).getValue(), fetchFieldValue + "_" + hit.docId()); - index++; - } - } - ); - } - - public void testRankFeaturePhaseExceptionOnCoordinatingNode() { - final String indexName = "index"; - final String rankFeatureFieldName = "field"; - final String searchFieldName = "search_field"; - final String searchFieldValue = "some_value"; - final String fetchFieldName = "fetch_field"; - final String fetchFieldValue = "fetch_value"; - - final int minDocs = 3; - final int maxDocs = 10; - int numDocs = between(minDocs, maxDocs); - createIndex(indexName); - // index some documents - for (int i = 0; i < numDocs; i++) { - prepareIndex(indexName).setId(String.valueOf(i)) - .setSource( - rankFeatureFieldName, - "aardvark_" + i, - searchFieldName, - searchFieldValue, - fetchFieldName, - fetchFieldValue + "_" + i - ) - .get(); - } - indicesAdmin().prepareRefresh(indexName).get(); - - expectThrows( - SearchPhaseExecutionException.class, - () -> client().prepareSearch(indexName) - .setSource( - new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) - .size(2) - .from(2) - .fetchField(fetchFieldName) - .rankBuilder(new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - - // no need for more than one queries - @Override - public boolean isCompoundBuilder() { - return false; - } - - @Override - public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( - int size, - int from, - Client client - ) { - return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { - @Override - protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { - throw new IllegalStateException("should have failed earlier"); - } - }; - } - - @Override - public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { - return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - @Override - public ScoreDoc[] rankQueryPhaseResults( - List querySearchResults, - SearchPhaseController.TopDocsStats topDocStats - ) { - throw new UnsupportedOperationException("simulated failure"); - } - }; - } - - @Override - public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { - return new QueryPhaseRankShardContext(queries, from) { - - @Override - public int rankWindowSize() { - return DEFAULT_RANK_WINDOW_SIZE; - } - - @Override - public RankShardResult combineQueryPhaseResults(List rankResults) { - // we know we have just 1 query, so return all the docs from it - return new TestRankShardResult( - Arrays.stream(rankResults.get(0).scoreDocs) - .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) - .limit(rankWindowSize()) - .toArray(RankDoc[]::new) - ); - } - }; - } - - @Override - public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { - return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { - @Override - public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { - RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; - for (int i = 0; i < hits.getHits().length; i++) { - SearchHit hit = hits.getHits()[i]; - rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); - rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); - rankFeatureDocs[i].score = randomFloat(); - rankFeatureDocs[i].rank = i + 1; - } - return new RankFeatureShardResult(rankFeatureDocs); - } - }; - } - }) - ) - .get() - ); - } - - public void testRankFeaturePhaseExceptionAllShardFail() { - final String indexName = "index"; - final String rankFeatureFieldName = "field"; - final String searchFieldName = "search_field"; - final String searchFieldValue = "some_value"; - final String fetchFieldName = "fetch_field"; - final String fetchFieldValue = "fetch_value"; - - final int minDocs = 3; - final int maxDocs = 10; - int numDocs = between(minDocs, maxDocs); - createIndex(indexName); - // index some documents - for (int i = 0; i < numDocs; i++) { - prepareIndex(indexName).setId(String.valueOf(i)) - .setSource( - rankFeatureFieldName, - "aardvark_" + i, - searchFieldName, - searchFieldValue, - fetchFieldName, - fetchFieldValue + "_" + i - ) - .get(); - } - indicesAdmin().prepareRefresh(indexName).get(); - - expectThrows( - SearchPhaseExecutionException.class, - () -> client().prepareSearch(indexName) - .setAllowPartialSearchResults(true) - .setSource( - new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) - .fetchField(fetchFieldName) - .rankBuilder( - // here we override only the shard-level contexts - new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - - // no need for more than one queries - @Override - public boolean isCompoundBuilder() { - return false; - } - - @Override - public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( - int size, - int from, - Client client - ) { - return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { - @Override - protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { - float[] scores = new float[featureDocs.length]; - for (int i = 0; i < featureDocs.length; i++) { - scores[i] = featureDocs[i].score; - } - scoreListener.onResponse(scores); - } - }; - } - - @Override - public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { - return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - @Override - public ScoreDoc[] rankQueryPhaseResults( - List querySearchResults, - SearchPhaseController.TopDocsStats topDocStats - ) { - List rankDocs = new ArrayList<>(); - for (int i = 0; i < querySearchResults.size(); i++) { - QuerySearchResult querySearchResult = querySearchResults.get(i); - TestRankShardResult shardResult = (TestRankShardResult) querySearchResult - .getRankShardResult(); - for (RankDoc trd : shardResult.testRankDocs) { - trd.shardIndex = i; - rankDocs.add(trd); - } - } - rankDocs.sort(Comparator.comparing((RankDoc doc) -> doc.score).reversed()); - RankDoc[] topResults = rankDocs.stream().limit(rankWindowSize).toArray(RankDoc[]::new); - topDocStats.fetchHits = topResults.length; - return topResults; - } - }; - } - - @Override - public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { - return new QueryPhaseRankShardContext(queries, from) { - - @Override - public int rankWindowSize() { - return DEFAULT_RANK_WINDOW_SIZE; - } - - @Override - public RankShardResult combineQueryPhaseResults(List rankResults) { - // we know we have just 1 query, so return all the docs from it - return new TestRankShardResult( - Arrays.stream(rankResults.get(0).scoreDocs) - .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) - .limit(rankWindowSize()) - .toArray(RankDoc[]::new) - ); - } - }; - } - - @Override - public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { - return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { - @Override - public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { - throw new UnsupportedOperationException("simulated failure"); - } - }; - } - } - ) - ) - .get() - ); - } - - public void testRankFeaturePhaseExceptionOneShardFails() { - // if we have only one shard and it fails, it will fallback to context.onPhaseFailure which will eventually clean up all contexts. - // in this test we want to make sure that even if one shard (of many) fails during the RankFeaturePhase, then the appropriate - // context will have been cleaned up. - final String indexName = "index"; - final String rankFeatureFieldName = "field"; - final String searchFieldName = "search_field"; - final String searchFieldValue = "some_value"; - final String fetchFieldName = "fetch_field"; - final String fetchFieldValue = "fetch_value"; - - final int minDocs = 3; - final int maxDocs = 10; - int numDocs = between(minDocs, maxDocs); - createIndex(indexName, Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 2).build()); - // index some documents - for (int i = 0; i < numDocs; i++) { - prepareIndex(indexName).setId(String.valueOf(i)) - .setSource( - rankFeatureFieldName, - "aardvark_" + i, - searchFieldName, - searchFieldValue, - fetchFieldName, - fetchFieldValue + "_" + i - ) - .get(); - } - indicesAdmin().prepareRefresh(indexName).get(); - - assertResponse( - client().prepareSearch(indexName) - .setAllowPartialSearchResults(true) - .setSource( - new SearchSourceBuilder().query(new TermQueryBuilder(searchFieldName, searchFieldValue)) - .fetchField(fetchFieldName) - .rankBuilder( - // here we override only the shard-level contexts - new TestRankBuilder(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - - // no need for more than one queries - @Override - public boolean isCompoundBuilder() { - return false; - } - - @Override - public RankFeaturePhaseRankCoordinatorContext buildRankFeaturePhaseCoordinatorContext( - int size, - int from, - Client client - ) { - return new RankFeaturePhaseRankCoordinatorContext(size, from, DEFAULT_RANK_WINDOW_SIZE) { - @Override - protected void computeScores(RankFeatureDoc[] featureDocs, ActionListener scoreListener) { - float[] scores = new float[featureDocs.length]; - for (int i = 0; i < featureDocs.length; i++) { - scores[i] = featureDocs[i].score; - } - scoreListener.onResponse(scores); - } - }; - } - - @Override - public QueryPhaseRankCoordinatorContext buildQueryPhaseCoordinatorContext(int size, int from) { - return new QueryPhaseRankCoordinatorContext(RankBuilder.DEFAULT_RANK_WINDOW_SIZE) { - @Override - public ScoreDoc[] rankQueryPhaseResults( - List querySearchResults, - SearchPhaseController.TopDocsStats topDocStats - ) { - List rankDocs = new ArrayList<>(); - for (int i = 0; i < querySearchResults.size(); i++) { - QuerySearchResult querySearchResult = querySearchResults.get(i); - TestRankShardResult shardResult = (TestRankShardResult) querySearchResult - .getRankShardResult(); - for (RankDoc trd : shardResult.testRankDocs) { - trd.shardIndex = i; - rankDocs.add(trd); - } - } - rankDocs.sort(Comparator.comparing((RankDoc doc) -> doc.score).reversed()); - RankDoc[] topResults = rankDocs.stream().limit(rankWindowSize).toArray(RankDoc[]::new); - topDocStats.fetchHits = topResults.length; - return topResults; - } - }; - } - - @Override - public QueryPhaseRankShardContext buildQueryPhaseShardContext(List queries, int from) { - return new QueryPhaseRankShardContext(queries, from) { - - @Override - public int rankWindowSize() { - return DEFAULT_RANK_WINDOW_SIZE; - } - - @Override - public RankShardResult combineQueryPhaseResults(List rankResults) { - // we know we have just 1 query, so return all the docs from it - return new TestRankShardResult( - Arrays.stream(rankResults.get(0).scoreDocs) - .map(x -> new RankDoc(x.doc, x.score, x.shardIndex)) - .limit(rankWindowSize()) - .toArray(RankDoc[]::new) - ); - } - }; - } - - @Override - public RankFeaturePhaseRankShardContext buildRankFeaturePhaseShardContext() { - return new RankFeaturePhaseRankShardContext(rankFeatureFieldName) { - @Override - public RankShardResult buildRankFeatureShardResult(SearchHits hits, int shardId) { - if (shardId == 0) { - throw new UnsupportedOperationException("simulated failure"); - } else { - RankFeatureDoc[] rankFeatureDocs = new RankFeatureDoc[hits.getHits().length]; - for (int i = 0; i < hits.getHits().length; i++) { - SearchHit hit = hits.getHits()[i]; - rankFeatureDocs[i] = new RankFeatureDoc(hit.docId(), hit.getScore(), shardId); - rankFeatureDocs[i].featureData(hit.getFields().get(rankFeatureFieldName).getValue()); - rankFeatureDocs[i].score = randomFloat(); - rankFeatureDocs[i].rank = i + 1; - } - return new RankFeatureShardResult(rankFeatureDocs); - } - } - }; - } - } - ) - ), - (searchResponse) -> { - assertEquals(1, searchResponse.getSuccessfulShards()); - assertEquals("simulated failure", searchResponse.getShardFailures()[0].getCause().getMessage()); - assertNotEquals(0, searchResponse.getHits().getHits().length); - for (SearchHit hit : searchResponse.getHits().getHits()) { - assertEquals(fetchFieldValue + "_" + hit.getId(), hit.getFields().get(fetchFieldName).getValue()); - assertEquals(1, hit.getShard().getShardId().id()); + recoverShardFromStore(indexShard); + assertTrue(indexDoc(indexShard, "_doc", "id", "{\"field\":\"value\"}").isCreated()); + assertTrue(indexShard.refresh("test").refreshed()); + try (Engine.Searcher searcher = indexShard.acquireSearcher("test")) { + SearchExecutionContext searchExecutionContext = createSearchExecutionContext( + (mappedFieldType, fieldDataContext) -> indexFieldData, + searcher + ); + SearchService.CanMatchContext canMatchContext = createCanMatchContext( + shardRequest, + indexShard, + searchExecutionContext, + parserConfig(), + throwException + ); + CanMatchShardResponse canMatchShardResponse = SearchService.canMatch(canMatchContext, false); + assertEquals(expectedCanMatch, canMatchShardResponse.canMatch()); + if (expectedMinAndMax == null) { + assertNull(canMatchShardResponse.estimatedMinAndMax()); + } else { + MinAndMax minAndMax = canMatchShardResponse.estimatedMinAndMax(); + assertNotNull(minAndMax); + assertEquals(expectedMinAndMax.getMin(), minAndMax.getMin()); + assertEquals(expectedMinAndMax.getMin(), minAndMax.getMax()); } - } - ); - } - public void testSearchWhileIndexDeletedDoesNotLeakSearchContext() throws ExecutionException, InterruptedException { - createIndex("index"); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - IndexShard indexShard = indexService.getShard(0); - - MockSearchService service = (MockSearchService) getInstanceFromNode(SearchService.class); - service.setOnPutContext(context -> { - if (context.indexShard() == indexShard) { - assertAcked(indicesAdmin().prepareDelete("index")); } - }); - - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - SearchRequest scrollSearchRequest = new SearchRequest().allowPartialSearchResults(true) - .scroll(new Scroll(TimeValue.timeValueMinutes(1))); - - // the scrolls are not explicitly freed, but should all be gone when the test finished. - // for completeness, we also randomly test the regular search path. - final boolean useScroll = randomBoolean(); - PlainActionFuture result = new PlainActionFuture<>(); - service.executeQueryPhase( - new ShardSearchRequest( - OriginalIndices.NONE, - useScroll ? scrollSearchRequest : searchRequest, - new ShardId(resolveIndex("index"), 0), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ), - new SearchShardTask(123L, "", "", "", null, emptyMap()), - result - ); - - try { - result.get(); - } catch (Exception e) { - // ok + } finally { + closeShards(indexShard); } - - expectThrows(IndexNotFoundException.class, () -> indicesAdmin().prepareGetIndex().setIndices("index").get()); - - assertEquals(0, service.getActiveContexts()); - - SearchStats.Stats totalStats = indexShard.searchStats().getTotal(); - assertEquals(0, totalStats.getQueryCurrent()); - assertEquals(0, totalStats.getScrollCurrent()); - assertEquals(0, totalStats.getFetchCurrent()); - } - - public void testBeforeShardLockDuringShardCreate() { - IndexService indexService = createIndex("index", Settings.builder().put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1).build()); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertResponse( - client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), - searchResponse -> assertThat(searchResponse.getScrollId(), is(notNullValue())) - ); - SearchService service = getInstanceFromNode(SearchService.class); - - assertEquals(1, service.getActiveContexts()); - service.beforeIndexShardCreated( - TestShardRouting.newShardRouting( - "test", - 0, - randomAlphaOfLength(5), - randomAlphaOfLength(5), - randomBoolean(), - ShardRoutingState.INITIALIZING - ), - indexService.getIndexSettings().getSettings() - ); - assertEquals(1, service.getActiveContexts()); - - service.beforeIndexShardCreated( - TestShardRouting.newShardRouting( - new ShardId(indexService.index(), 0), - randomAlphaOfLength(5), - randomBoolean(), - ShardRoutingState.INITIALIZING - ), - indexService.getIndexSettings().getSettings() - ); - assertEquals(0, service.getActiveContexts()); } - public void testTimeout() throws IOException { - createIndex("index"); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - final ShardSearchRequest requestWithDefaultTimeout = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), + private SearchExecutionContext createSearchExecutionContext( + BiFunction> indexFieldDataLookup, + IndexSearcher searcher + ) { + IndexMetadata indexMetadata = IndexMetadata.builder("index") + .settings(Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, IndexVersion.current())) + .numberOfShards(1) + .numberOfReplicas(0) + .build(); + IndexSettings indexSettings = new IndexSettings(indexMetadata, Settings.EMPTY); + Predicate indexNameMatcher = pattern -> Regex.simpleMatch(pattern, "index"); + + MapperBuilderContext root = MapperBuilderContext.root(false, false); + RootObjectMapper.Builder builder = new RootObjectMapper.Builder("_doc", ObjectMapper.Defaults.SUBOBJECTS); + Mapping mapping = new Mapping( + builder.build(MapperBuilderContext.root(false, false)), + new MetadataFieldMapper[0], + Collections.emptyMap() + ); + KeywordFieldMapper keywordFieldMapper = new KeywordFieldMapper.Builder("field", IndexVersion.current()).build(root); + MappingLookup mappingLookup = MappingLookup.fromMappers( + mapping, + Collections.singletonList(keywordFieldMapper), + Collections.emptyList() + ); + return new SearchExecutionContext( 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - - try ( - ReaderContext reader = createReaderContext(indexService, indexShard); - SearchContext contextWithDefaultTimeout = service.createContext( - reader, - requestWithDefaultTimeout, - mock(SearchShardTask.class), - ResultsType.NONE, - randomBoolean() - ) - ) { - // the search context should inherit the default timeout - assertThat(contextWithDefaultTimeout.timeout(), equalTo(TimeValue.timeValueSeconds(5))); - } - - final long seconds = randomIntBetween(6, 10); - searchRequest.source(new SearchSourceBuilder().timeout(TimeValue.timeValueSeconds(seconds))); - final ShardSearchRequest requestWithCustomTimeout = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null + indexSettings, + null, + indexFieldDataLookup, + null, + mappingLookup, + null, + null, + parserConfig(), + writableRegistry(), + null, + searcher, + System::currentTimeMillis, + null, + indexNameMatcher, + () -> true, + null, + Collections.emptyMap(), + MapperMetrics.NOOP ); - try ( - ReaderContext reader = createReaderContext(indexService, indexShard); - SearchContext context = service.createContext( - reader, - requestWithCustomTimeout, - mock(SearchShardTask.class), - ResultsType.NONE, - randomBoolean() - ) - ) { - // the search context should inherit the query timeout - assertThat(context.timeout(), equalTo(TimeValue.timeValueSeconds(seconds))); - } } - /** - * test that getting more than the allowed number of docvalue_fields throws an exception - */ - public void testMaxDocvalueFieldsSearch() throws IOException { - final Settings settings = Settings.builder().put(IndexSettings.MAX_DOCVALUE_FIELDS_SEARCH_SETTING.getKey(), 1).build(); - createIndex("index", settings, null, "field1", "keyword", "field2", "keyword"); - prepareIndex("index").setId("1").setSource("field1", "value1", "field2", "value2").setRefreshPolicy(IMMEDIATE).get(); - - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchRequest.source(searchSourceBuilder); - searchSourceBuilder.docValueField("field1"); - - final ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - try ( - ReaderContext reader = createReaderContext(indexService, indexShard); - SearchContext context = service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) - ) { - assertNotNull(context); - } - - searchSourceBuilder.docValueField("unmapped_field"); - try ( - ReaderContext reader = createReaderContext(indexService, indexShard); - SearchContext context = service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) - ) { - assertNotNull(context); - } - - searchSourceBuilder.docValueField("field2"); - try (ReaderContext reader = createReaderContext(indexService, indexShard)) { - IllegalArgumentException ex = expectThrows( - IllegalArgumentException.class, - () -> service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) - ); - assertEquals( - "Trying to retrieve too many docvalue_fields. Must be less than or equal to: [1] but was [2]. " - + "This limit can be set by changing the [index.max_docvalue_fields_search] index level setting.", - ex.getMessage() - ); - } - } + private static IndexFieldData indexFieldData(SortField sortField) { + return new IndexFieldData<>() { + @Override + public String getFieldName() { + return "field"; + } - public void testDeduplicateDocValuesFields() throws Exception { - createIndex("index", Settings.EMPTY, "_doc", "field1", "type=date", "field2", "type=date"); - prepareIndex("index").setId("1").setSource("field1", "2022-08-03", "field2", "2022-08-04").setRefreshPolicy(IMMEDIATE).get(); - SearchService service = getInstanceFromNode(SearchService.class); - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - IndexShard indexShard = indexService.getShard(0); + @Override + public ValuesSourceType getValuesSourceType() { + throw new UnsupportedOperationException(); + } - try (ReaderContext reader = createReaderContext(indexService, indexShard)) { - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchRequest.source(searchSourceBuilder); - searchSourceBuilder.docValueField("f*"); - if (randomBoolean()) { - searchSourceBuilder.docValueField("field*"); + @Override + public LeafFieldData load(LeafReaderContext context) { + throw new UnsupportedOperationException(); } - if (randomBoolean()) { - searchSourceBuilder.docValueField("*2"); + + @Override + public LeafFieldData loadDirect(LeafReaderContext context) { + throw new UnsupportedOperationException(); } - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - try ( - SearchContext context = service.createContext( - reader, - request, - mock(SearchShardTask.class), - ResultsType.NONE, - randomBoolean() - ) + + @Override + public SortField sortField( + Object missingValue, + MultiValueMode sortMode, + XFieldComparatorSource.Nested nested, + boolean reverse ) { - Collection fields = context.docValuesContext().fields(); - assertThat(fields, containsInAnyOrder(new FieldAndFormat("field1", null), new FieldAndFormat("field2", null))); + return sortField; } - } - } - - /** - * test that getting more than the allowed number of script_fields throws an exception - */ - public void testMaxScriptFieldsSearch() throws IOException { - createIndex("index"); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchRequest.source(searchSourceBuilder); - // adding the maximum allowed number of script_fields to retrieve - int maxScriptFields = indexService.getIndexSettings().getMaxScriptFields(); - for (int i = 0; i < maxScriptFields; i++) { - searchSourceBuilder.scriptField( - "field" + i, - new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, emptyMap()) - ); - } - final ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - try (ReaderContext reader = createReaderContext(indexService, indexShard)) { - try ( - SearchContext context = service.createContext( - reader, - request, - mock(SearchShardTask.class), - ResultsType.NONE, - randomBoolean() - ) + @Override + public BucketedSort newBucketedSort( + BigArrays bigArrays, + Object missingValue, + MultiValueMode sortMode, + XFieldComparatorSource.Nested nested, + SortOrder sortOrder, + DocValueFormat format, + int bucketSize, + BucketedSort.ExtraData extra ) { - assertNotNull(context); + throw new UnsupportedOperationException(); } - searchSourceBuilder.scriptField( - "anotherScriptField", - new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, emptyMap()) - ); - IllegalArgumentException ex = expectThrows( - IllegalArgumentException.class, - () -> service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) - ); - assertEquals( - "Trying to retrieve too many script_fields. Must be less than or equal to: [" - + maxScriptFields - + "] but was [" - + (maxScriptFields + 1) - + "]. This limit can be set by changing the [index.max_script_fields] index level setting.", - ex.getMessage() - ); - } - } - - public void testIgnoreScriptfieldIfSizeZero() throws IOException { - createIndex("index"); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - SearchSourceBuilder searchSourceBuilder = new SearchSourceBuilder(); - searchRequest.source(searchSourceBuilder); - searchSourceBuilder.scriptField( - "field" + 0, - new Script(ScriptType.INLINE, MockScriptEngine.NAME, CustomScriptPlugin.DUMMY_SCRIPT, emptyMap()) - ); - searchSourceBuilder.size(0); - final ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - try ( - ReaderContext reader = createReaderContext(indexService, indexShard); - SearchContext context = service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) - ) { - assertEquals(0, context.scriptFields().fields().size()); - } + }; } - /** - * test that creating more than the allowed number of scroll contexts throws an exception - */ - public void testMaxOpenScrollContexts() throws Exception { - createIndex("index"); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - - // Open all possible scrolls, clear some of them, then open more until the limit is reached - LinkedList clearScrollIds = new LinkedList<>(); - - for (int i = 0; i < SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY); i++) { - assertResponse(client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)), searchResponse -> { - if (randomInt(4) == 0) clearScrollIds.addLast(searchResponse.getScrollId()); - }); - } + private static SearchService.CanMatchContext createCanMatchContext( + ShardSearchRequest shardRequest, + IndexShard indexShard, + SearchExecutionContext searchExecutionContext, + XContentParserConfiguration parserConfig, + boolean throwException + ) { + return new SearchService.CanMatchContext(shardRequest, null, null, -1, -1) { + @Override + IndexShard getShard() { + return indexShard; + } - ClearScrollRequest clearScrollRequest = new ClearScrollRequest(); - clearScrollRequest.setScrollIds(clearScrollIds); - client().clearScroll(clearScrollRequest).get(); + @Override + QueryRewriteContext getQueryRewriteContext(IndexService indexService) { + if (throwException) { + throw new IllegalArgumentException(); + } + return new QueryRewriteContext(parserConfig, null, System::currentTimeMillis); + } - for (int i = 0; i < clearScrollIds.size(); i++) { - client().prepareSearch("index").setSize(1).setScroll(TimeValue.timeValueMinutes(1)).get().decRef(); - } + @Override + SearchExecutionContext getSearchExecutionContext(Engine.Searcher searcher) { + return searchExecutionContext; + } - final ShardScrollRequestTest request = new ShardScrollRequestTest(indexShard.shardId()); - ElasticsearchException ex = expectThrows( - ElasticsearchException.class, - () -> service.createAndPutReaderContext( - request, - indexService, - indexShard, - indexShard.acquireSearcherSupplier(), - SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() - ) - ); - assertEquals( - "Trying to create too many scroll contexts. Must be less than or equal to: [" - + SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY) - + "]. " - + "This limit can be set by changing the [search.max_open_scroll_context] setting.", - ex.getMessage() - ); - assertEquals(RestStatus.TOO_MANY_REQUESTS, ex.status()); - - service.freeAllScrollContexts(); - } - - public void testOpenScrollContextsConcurrently() throws Exception { - createIndex("index"); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - - final int maxScrollContexts = SearchService.MAX_OPEN_SCROLL_CONTEXT.get(Settings.EMPTY); - final SearchService searchService = getInstanceFromNode(SearchService.class); - Thread[] threads = new Thread[randomIntBetween(2, 8)]; - CountDownLatch latch = new CountDownLatch(threads.length); - for (int i = 0; i < threads.length; i++) { - threads[i] = new Thread(() -> { - latch.countDown(); - try { - latch.await(); - for (;;) { - final Engine.SearcherSupplier reader = indexShard.acquireSearcherSupplier(); - try { - final ShardScrollRequestTest request = new ShardScrollRequestTest(indexShard.shardId()); - searchService.createAndPutReaderContext( - request, - indexService, - indexShard, - reader, - SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() - ); - } catch (ElasticsearchException e) { - assertThat( - e.getMessage(), - equalTo( - "Trying to create too many scroll contexts. Must be less than or equal to: " - + "[" - + maxScrollContexts - + "]. " - + "This limit can be set by changing the [search.max_open_scroll_context] setting." - ) - ); - return; - } - } - } catch (Exception e) { - throw new AssertionError(e); - } - }); - threads[i].setName("elasticsearch[node_s_0][search]"); - threads[i].start(); - } - for (Thread thread : threads) { - thread.join(); - } - assertThat(searchService.getActiveContexts(), equalTo(maxScrollContexts)); - searchService.freeAllScrollContexts(); - } - - public static class FailOnRewriteQueryPlugin extends Plugin implements SearchPlugin { - @Override - public List> getQueries() { - return singletonList(new QuerySpec<>("fail_on_rewrite_query", FailOnRewriteQueryBuilder::new, parseContext -> { - throw new UnsupportedOperationException("No query parser for this plugin"); - })); - } - } - - public static class FailOnRewriteQueryBuilder extends DummyQueryBuilder { - - public FailOnRewriteQueryBuilder(StreamInput in) throws IOException { - super(in); - } - - public FailOnRewriteQueryBuilder() {} - - @Override - protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) { - if (queryRewriteContext.convertToSearchExecutionContext() != null) { - throw new IllegalStateException("Fail on rewrite phase"); - } - return this; - } - } - - private static class ShardScrollRequestTest extends ShardSearchRequest { - private Scroll scroll; - - ShardScrollRequestTest(ShardId shardId) { - super( - OriginalIndices.NONE, - new SearchRequest().allowPartialSearchResults(true), - shardId, - 0, - 1, - AliasFilter.EMPTY, - 1f, - -1, - null - ); - this.scroll = new Scroll(TimeValue.timeValueMinutes(1)); - } - - @Override - public Scroll scroll() { - return this.scroll; - } - } - - public void testCanMatch() throws Exception { - createIndex("index"); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - assertTrue( - service.canMatch( - new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) - ).canMatch() - ); - - searchRequest.source(new SearchSourceBuilder()); - assertTrue( - service.canMatch( - new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) - ).canMatch() - ); - - searchRequest.source(new SearchSourceBuilder().query(new MatchAllQueryBuilder())); - assertTrue( - service.canMatch( - new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) - ).canMatch() - ); - - searchRequest.source( - new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) - .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(0)) - ); - assertTrue( - service.canMatch( - new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) - ).canMatch() - ); - searchRequest.source( - new SearchSourceBuilder().query(new MatchNoneQueryBuilder()).aggregation(new GlobalAggregationBuilder("test")) - ); - assertTrue( - service.canMatch( - new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) - ).canMatch() - ); - - searchRequest.source(new SearchSourceBuilder().query(new MatchNoneQueryBuilder())); - assertFalse( - service.canMatch( - new ShardSearchRequest(OriginalIndices.NONE, searchRequest, indexShard.shardId(), 0, 1, AliasFilter.EMPTY, 1f, -1, null) - ).canMatch() - ); - assertEquals(5, numWrapInvocations.get()); - - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - - /* - * Checks that canMatch takes into account the alias filter - */ - // the source cannot be rewritten to a match_none - searchRequest.indices("alias").source(new SearchSourceBuilder().query(new MatchAllQueryBuilder())); - assertFalse( - service.canMatch( - new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.of(new TermQueryBuilder("foo", "bar"), "alias"), - 1f, - -1, - null - ) - ).canMatch() - ); - // the source can match and can be rewritten to a match_none, but not the alias filter - final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); - assertEquals(RestStatus.CREATED, response.status()); - searchRequest.indices("alias").source(new SearchSourceBuilder().query(new TermQueryBuilder("id", "1"))); - assertFalse( - service.canMatch( - new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.of(new TermQueryBuilder("foo", "bar"), "alias"), - 1f, - -1, - null - ) - ).canMatch() - ); - - CountDownLatch latch = new CountDownLatch(1); - SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); - // Because the foo field used in alias filter is unmapped the term query builder rewrite can resolve to a match no docs query, - // without acquiring a searcher and that means the wrapper is not called - assertEquals(5, numWrapInvocations.get()); - service.executeQueryPhase(request, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult searchPhaseResult) { - try { - // make sure that the wrapper is called when the query is actually executed - assertEquals(6, numWrapInvocations.get()); - } finally { - latch.countDown(); - } - } - - @Override - public void onFailure(Exception e) { - try { - throw new AssertionError(e); - } finally { - latch.countDown(); - } - } - }); - latch.await(); - } - - public void testCanRewriteToMatchNone() { - assertFalse( - SearchService.canRewriteToMatchNone( - new SearchSourceBuilder().query(new MatchNoneQueryBuilder()).aggregation(new GlobalAggregationBuilder("test")) - ) - ); - assertFalse(SearchService.canRewriteToMatchNone(new SearchSourceBuilder())); - assertFalse(SearchService.canRewriteToMatchNone(null)); - assertFalse( - SearchService.canRewriteToMatchNone( - new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) - .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(0)) - ) - ); - assertTrue(SearchService.canRewriteToMatchNone(new SearchSourceBuilder().query(new TermQueryBuilder("foo", "bar")))); - assertTrue( - SearchService.canRewriteToMatchNone( - new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) - .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(1)) - ) - ); - assertFalse( - SearchService.canRewriteToMatchNone( - new SearchSourceBuilder().query(new MatchNoneQueryBuilder()) - .aggregation(new TermsAggregationBuilder("test").userValueTypeHint(ValueType.STRING).minDocCount(1)) - .suggest(new SuggestBuilder()) - ) - ); - assertFalse( - SearchService.canRewriteToMatchNone( - new SearchSourceBuilder().query(new TermQueryBuilder("foo", "bar")).suggest(new SuggestBuilder()) - ) - ); - } - - public void testSetSearchThrottled() throws IOException { - createIndex("throttled_threadpool_index"); - client().execute( - InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, - new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( - "throttled_threadpool_index", - IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), - "true" - ) - ).actionGet(); - final SearchService service = getInstanceFromNode(SearchService.class); - Index index = resolveIndex("throttled_threadpool_index"); - assertTrue(service.getIndicesService().indexServiceSafe(index).getIndexSettings().isSearchThrottled()); - prepareIndex("throttled_threadpool_index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertSearchHits( - client().prepareSearch("throttled_threadpool_index") - .setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED) - .setSize(1), - "1" - ); - // we add a search action listener in a plugin above to assert that this is actually used - client().execute( - InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, - new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( - "throttled_threadpool_index", - IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), - "false" - ) - ).actionGet(); - - IllegalArgumentException iae = expectThrows( - IllegalArgumentException.class, - () -> indicesAdmin().prepareUpdateSettings("throttled_threadpool_index") - .setSettings(Settings.builder().put(IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), false)) - .get() - ); - assertEquals("can not update private setting [index.search.throttled]; this setting is managed by Elasticsearch", iae.getMessage()); - assertFalse(service.getIndicesService().indexServiceSafe(index).getIndexSettings().isSearchThrottled()); - } - - public void testAggContextGetsMatchAll() throws IOException { - createIndex("test"); - withAggregationContext("test", context -> assertThat(context.query(), equalTo(new MatchAllDocsQuery()))); - } - - public void testAggContextGetsNestedFilter() throws IOException { - XContentBuilder mapping = JsonXContent.contentBuilder().startObject().startObject("properties"); - mapping.startObject("nested").field("type", "nested").endObject(); - mapping.endObject().endObject(); - - createIndex("test", Settings.EMPTY, mapping); - withAggregationContext("test", context -> assertThat(context.query(), equalTo(new MatchAllDocsQuery()))); - } - - /** - * Build an {@link AggregationContext} with the named index. - */ - private void withAggregationContext(String index, Consumer check) throws IOException { - IndexService indexService = getInstanceFromNode(IndicesService.class).indexServiceSafe(resolveIndex(index)); - ShardId shardId = new ShardId(indexService.index(), 0); - - SearchRequest request = new SearchRequest().indices(index) - .source(new SearchSourceBuilder().aggregation(new FiltersAggregationBuilder("test", new MatchAllQueryBuilder()))) - .allowPartialSearchResults(false); - ShardSearchRequest shardRequest = new ShardSearchRequest( - OriginalIndices.NONE, - request, - shardId, - 0, - 1, - AliasFilter.EMPTY, - 1, - 0, - null - ); - - try (ReaderContext readerContext = createReaderContext(indexService, indexService.getShard(0))) { - try ( - SearchContext context = getInstanceFromNode(SearchService.class).createContext( - readerContext, - shardRequest, - mock(SearchShardTask.class), - ResultsType.QUERY, - true - ) - ) { - check.accept(context.aggregations().factories().context()); - } - } - } - - public void testExpandSearchThrottled() { - createIndex("throttled_threadpool_index"); - client().execute( - InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, - new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request( - "throttled_threadpool_index", - IndexSettings.INDEX_SEARCH_THROTTLED.getKey(), - "true" - ) - ).actionGet(); - - prepareIndex("throttled_threadpool_index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertHitCount(client().prepareSearch(), 1L); - assertHitCount(client().prepareSearch().setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED), 1L); - } - - public void testExpandSearchFrozen() { - String indexName = "frozen_index"; - createIndex(indexName); - client().execute( - InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.INSTANCE, - new InternalOrPrivateSettingsPlugin.UpdateInternalOrPrivateAction.Request(indexName, "index.frozen", "true") - ).actionGet(); - - prepareIndex(indexName).setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - assertHitCount(client().prepareSearch(), 0L); - assertHitCount(client().prepareSearch().setIndicesOptions(IndicesOptions.STRICT_EXPAND_OPEN_FORBID_CLOSED), 1L); - assertWarnings(TransportSearchAction.FROZEN_INDICES_DEPRECATION_MESSAGE.replace("{}", indexName)); - } - - public void testCreateReduceContext() { - SearchService service = getInstanceFromNode(SearchService.class); - AggregationReduceContext.Builder reduceContextBuilder = service.aggReduceContextBuilder( - () -> false, - new SearchRequest().source(new SearchSourceBuilder()).source().aggregations() - ); - { - AggregationReduceContext reduceContext = reduceContextBuilder.forFinalReduction(); - expectThrows( - MultiBucketConsumerService.TooManyBucketsException.class, - () -> reduceContext.consumeBucketsAndMaybeBreak(MultiBucketConsumerService.DEFAULT_MAX_BUCKETS + 1) - ); - } - { - AggregationReduceContext reduceContext = reduceContextBuilder.forPartialReduction(); - reduceContext.consumeBucketsAndMaybeBreak(MultiBucketConsumerService.DEFAULT_MAX_BUCKETS + 1); - } - } - - public void testMultiBucketConsumerServiceCB() { - MultiBucketConsumerService service = new MultiBucketConsumerService( - getInstanceFromNode(ClusterService.class), - Settings.EMPTY, - new NoopCircuitBreaker("test") { - - @Override - public void addEstimateBytesAndMaybeBreak(long bytes, String label) throws CircuitBreakingException { - throw new CircuitBreakingException("tripped", getDurability()); - } - } - ); - // for partial - { - IntConsumer consumer = service.createForPartial(); - for (int i = 0; i < 1023; i++) { - consumer.accept(0); - } - CircuitBreakingException ex = expectThrows(CircuitBreakingException.class, () -> consumer.accept(0)); - assertThat(ex.getMessage(), equalTo("tripped")); - } - // for final - { - IntConsumer consumer = service.createForFinal(); - for (int i = 0; i < 1023; i++) { - consumer.accept(0); - } - CircuitBreakingException ex = expectThrows(CircuitBreakingException.class, () -> consumer.accept(0)); - assertThat(ex.getMessage(), equalTo("tripped")); - } - } - - public void testCreateSearchContext() throws IOException { - String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); - IndexService indexService = createIndex(index); - final SearchService service = getInstanceFromNode(SearchService.class); - ShardId shardId = new ShardId(indexService.index(), 0); - long nowInMillis = System.currentTimeMillis(); - String clusterAlias = randomBoolean() ? null : randomAlphaOfLengthBetween(3, 10); - SearchRequest searchRequest = new SearchRequest(); - searchRequest.allowPartialSearchResults(randomBoolean()); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - shardId, - 0, - indexService.numberOfShards(), - AliasFilter.EMPTY, - 1f, - nowInMillis, - clusterAlias - ); - try (SearchContext searchContext = service.createSearchContext(request, new TimeValue(System.currentTimeMillis()))) { - SearchShardTarget searchShardTarget = searchContext.shardTarget(); - SearchExecutionContext searchExecutionContext = searchContext.getSearchExecutionContext(); - String expectedIndexName = clusterAlias == null ? index : clusterAlias + ":" + index; - assertEquals(expectedIndexName, searchExecutionContext.getFullyQualifiedIndex().getName()); - assertEquals(expectedIndexName, searchShardTarget.getFullyQualifiedIndexName()); - assertEquals(clusterAlias, searchShardTarget.getClusterAlias()); - assertEquals(shardId, searchShardTarget.getShardId()); - - assertNull(searchContext.dfsResult()); - searchContext.addDfsResult(); - assertSame(searchShardTarget, searchContext.dfsResult().getSearchShardTarget()); - - assertNull(searchContext.queryResult()); - searchContext.addQueryResult(); - assertSame(searchShardTarget, searchContext.queryResult().getSearchShardTarget()); - - assertNull(searchContext.fetchResult()); - searchContext.addFetchResult(); - assertSame(searchShardTarget, searchContext.fetchResult().getSearchShardTarget()); - } - } - - /** - * While we have no NPE in DefaultContext constructor anymore, we still want to guard against it (or other failures) in the future to - * avoid leaking searchers. - */ - public void testCreateSearchContextFailure() throws Exception { - final String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); - final IndexService indexService = createIndex(index); - final SearchService service = getInstanceFromNode(SearchService.class); - final ShardId shardId = new ShardId(indexService.index(), 0); - final ShardSearchRequest request = new ShardSearchRequest(shardId, 0, null) { @Override - public SearchType searchType() { - // induce an artificial NPE - throw new NullPointerException("expected"); + IndexService getIndexService() { + // it's ok to return null because the three above methods are overridden + return null; } }; - try (ReaderContext reader = createReaderContext(indexService, indexService.getShard(shardId.id()))) { - NullPointerException e = expectThrows( - NullPointerException.class, - () -> service.createContext(reader, request, mock(SearchShardTask.class), ResultsType.NONE, randomBoolean()) - ); - assertEquals("expected", e.getMessage()); - } - // Needs to busily assert because Engine#refreshNeeded can increase the refCount. - assertBusy( - () -> assertEquals("should have 2 store refs (IndexService + InternalEngine)", 2, indexService.getShard(0).store().refCount()) - ); - } - - public void testMatchNoDocsEmptyResponse() throws InterruptedException { - createIndex("index"); - Thread currentThread = Thread.currentThread(); - SearchService service = getInstanceFromNode(SearchService.class); - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(false) - .source(new SearchSourceBuilder().aggregation(AggregationBuilders.count("count").field("value"))); - ShardSearchRequest shardRequest = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 5, - AliasFilter.EMPTY, - 1.0f, - 0, - null - ); - SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); - - { - CountDownLatch latch = new CountDownLatch(1); - shardRequest.source().query(new MatchAllQueryBuilder()); - service.executeQueryPhase(shardRequest, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult result) { - try { - assertNotSame(Thread.currentThread(), currentThread); - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); - assertThat(result, instanceOf(QuerySearchResult.class)); - assertFalse(result.queryResult().isNull()); - assertNotNull(result.queryResult().topDocs()); - assertNotNull(result.queryResult().aggregations()); - } finally { - latch.countDown(); - } - } - - @Override - public void onFailure(Exception exc) { - try { - throw new AssertionError(exc); - } finally { - latch.countDown(); - } - } - }); - latch.await(); - } - - { - CountDownLatch latch = new CountDownLatch(1); - shardRequest.source().query(new MatchNoneQueryBuilder()); - service.executeQueryPhase(shardRequest, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult result) { - try { - assertNotSame(Thread.currentThread(), currentThread); - assertThat(Thread.currentThread().getName(), startsWith("elasticsearch[node_s_0][search]")); - assertThat(result, instanceOf(QuerySearchResult.class)); - assertFalse(result.queryResult().isNull()); - assertNotNull(result.queryResult().topDocs()); - assertNotNull(result.queryResult().aggregations()); - } finally { - latch.countDown(); - } - } - - @Override - public void onFailure(Exception exc) { - try { - throw new AssertionError(exc); - } finally { - latch.countDown(); - } - } - }); - latch.await(); - } - - { - CountDownLatch latch = new CountDownLatch(1); - shardRequest.canReturnNullResponseIfMatchNoDocs(true); - service.executeQueryPhase(shardRequest, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult result) { - try { - // make sure we don't use the search threadpool - assertSame(Thread.currentThread(), currentThread); - assertThat(result, instanceOf(QuerySearchResult.class)); - assertTrue(result.queryResult().isNull()); - } finally { - latch.countDown(); - } - } - - @Override - public void onFailure(Exception e) { - try { - throw new AssertionError(e); - } finally { - latch.countDown(); - } - } - }); - latch.await(); - } - } - - public void testDeleteIndexWhileSearch() throws Exception { - createIndex("test"); - int numDocs = randomIntBetween(1, 20); - for (int i = 0; i < numDocs; i++) { - prepareIndex("test").setSource("f", "v").get(); - } - indicesAdmin().prepareRefresh("test").get(); - AtomicBoolean stopped = new AtomicBoolean(false); - Thread[] searchers = new Thread[randomIntBetween(1, 4)]; - CountDownLatch latch = new CountDownLatch(searchers.length); - for (int i = 0; i < searchers.length; i++) { - searchers[i] = new Thread(() -> { - latch.countDown(); - while (stopped.get() == false) { - try { - client().prepareSearch("test").setRequestCache(false).get().decRef(); - } catch (Exception ignored) { - return; - } - } - }); - searchers[i].start(); - } - latch.await(); - indicesAdmin().prepareDelete("test").get(); - stopped.set(true); - for (Thread searcher : searchers) { - searcher.join(); - } - } - - public void testLookUpSearchContext() throws Exception { - createIndex("index"); - SearchService searchService = getInstanceFromNode(SearchService.class); - IndicesService indicesService = getInstanceFromNode(IndicesService.class); - IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - IndexShard indexShard = indexService.getShard(0); - List contextIds = new ArrayList<>(); - int numContexts = randomIntBetween(1, 10); - CountDownLatch latch = new CountDownLatch(1); - indexShard.getThreadPool().executor(ThreadPool.Names.SEARCH).execute(() -> { - try { - for (int i = 0; i < numContexts; i++) { - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - new SearchRequest().allowPartialSearchResults(true), - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - final ReaderContext context = searchService.createAndPutReaderContext( - request, - indexService, - indexShard, - indexShard.acquireSearcherSupplier(), - SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() - ); - assertThat(context.id().getId(), equalTo((long) (i + 1))); - contextIds.add(context.id()); - } - assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); - while (contextIds.isEmpty() == false) { - final ShardSearchContextId contextId = randomFrom(contextIds); - assertFalse(searchService.freeReaderContext(new ShardSearchContextId(UUIDs.randomBase64UUID(), contextId.getId()))); - assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); - if (randomBoolean()) { - assertTrue(searchService.freeReaderContext(contextId)); - } else { - assertTrue( - searchService.freeReaderContext((new ShardSearchContextId(contextId.getSessionId(), contextId.getId()))) - ); - } - contextIds.remove(contextId); - assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); - assertFalse(searchService.freeReaderContext(contextId)); - assertThat(searchService.getActiveContexts(), equalTo(contextIds.size())); - } - } finally { - latch.countDown(); - } - }); - latch.await(); - } - - public void testOpenReaderContext() { - createIndex("index"); - SearchService searchService = getInstanceFromNode(SearchService.class); - PlainActionFuture future = new PlainActionFuture<>(); - searchService.openReaderContext(new ShardId(resolveIndex("index"), 0), TimeValue.timeValueMinutes(between(1, 10)), future); - future.actionGet(); - assertThat(searchService.getActiveContexts(), equalTo(1)); - assertTrue(searchService.freeReaderContext(future.actionGet())); - } - - public void testCancelQueryPhaseEarly() throws Exception { - createIndex("index"); - final MockSearchService service = (MockSearchService) getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - - CountDownLatch latch1 = new CountDownLatch(1); - SearchShardTask task = new SearchShardTask(1, "", "", "", TaskId.EMPTY_TASK_ID, emptyMap()); - service.executeQueryPhase(request, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult searchPhaseResult) { - service.freeReaderContext(searchPhaseResult.getContextId()); - latch1.countDown(); - } - - @Override - public void onFailure(Exception e) { - try { - fail("Search should not be cancelled"); - } finally { - latch1.countDown(); - } - } - }); - latch1.await(); - - CountDownLatch latch2 = new CountDownLatch(1); - service.executeDfsPhase(request, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult searchPhaseResult) { - service.freeReaderContext(searchPhaseResult.getContextId()); - latch2.countDown(); - } - - @Override - public void onFailure(Exception e) { - try { - fail("Search should not be cancelled"); - } finally { - latch2.countDown(); - } - } - }); - latch2.await(); - - AtomicBoolean searchContextCreated = new AtomicBoolean(false); - service.setOnCreateSearchContext(c -> searchContextCreated.set(true)); - CountDownLatch latch3 = new CountDownLatch(1); - TaskCancelHelper.cancel(task, "simulated"); - service.executeQueryPhase(request, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult searchPhaseResult) { - try { - fail("Search not cancelled early"); - } finally { - service.freeReaderContext(searchPhaseResult.getContextId()); - searchPhaseResult.decRef(); - latch3.countDown(); - } - } - - @Override - public void onFailure(Exception e) { - assertThat(e, is(instanceOf(TaskCancelledException.class))); - assertThat(e.getMessage(), is("task cancelled [simulated]")); - assertThat(((TaskCancelledException) e).status(), is(RestStatus.BAD_REQUEST)); - assertThat(searchContextCreated.get(), is(false)); - latch3.countDown(); - } - }); - latch3.await(); - - searchContextCreated.set(false); - CountDownLatch latch4 = new CountDownLatch(1); - service.executeDfsPhase(request, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult searchPhaseResult) { - try { - fail("Search not cancelled early"); - } finally { - service.freeReaderContext(searchPhaseResult.getContextId()); - latch4.countDown(); - } - } - - @Override - public void onFailure(Exception e) { - assertThat(e, is(instanceOf(TaskCancelledException.class))); - assertThat(e.getMessage(), is("task cancelled [simulated]")); - assertThat(((TaskCancelledException) e).status(), is(RestStatus.BAD_REQUEST)); - assertThat(searchContextCreated.get(), is(false)); - latch4.countDown(); - } - }); - latch4.await(); - } - - public void testCancelFetchPhaseEarly() throws Exception { - createIndex("index"); - final MockSearchService service = (MockSearchService) getInstanceFromNode(SearchService.class); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - - AtomicBoolean searchContextCreated = new AtomicBoolean(false); - service.setOnCreateSearchContext(c -> searchContextCreated.set(true)); - - // Test fetch phase is cancelled early - String scrollId; - var searchResponse = client().search(searchRequest.allowPartialSearchResults(false).scroll(TimeValue.timeValueMinutes(10))).get(); - try { - scrollId = searchResponse.getScrollId(); - } finally { - searchResponse.decRef(); - } - - client().searchScroll(new SearchScrollRequest(scrollId)).get().decRef(); - assertThat(searchContextCreated.get(), is(true)); - - ClearScrollRequest clearScrollRequest = new ClearScrollRequest(); - clearScrollRequest.addScrollId(scrollId); - client().clearScroll(clearScrollRequest); - - searchResponse = client().search(searchRequest.allowPartialSearchResults(false).scroll(TimeValue.timeValueMinutes(10))).get(); - try { - scrollId = searchResponse.getScrollId(); - } finally { - searchResponse.decRef(); - } - searchContextCreated.set(false); - service.setOnCheckCancelled(t -> { - SearchShardTask task = new SearchShardTask(randomLong(), "transport", "action", "", TaskId.EMPTY_TASK_ID, emptyMap()); - TaskCancelHelper.cancel(task, "simulated"); - return task; - }); - CountDownLatch latch = new CountDownLatch(1); - client().searchScroll(new SearchScrollRequest(scrollId), new ActionListener<>() { - @Override - public void onResponse(SearchResponse searchResponse) { - try { - fail("Search not cancelled early"); - } finally { - latch.countDown(); - } - } - - @Override - public void onFailure(Exception e) { - Throwable cancelledExc = e.getCause().getCause(); - assertThat(cancelledExc, is(instanceOf(TaskCancelledException.class))); - assertThat(cancelledExc.getMessage(), is("task cancelled [simulated]")); - assertThat(((TaskCancelledException) cancelledExc).status(), is(RestStatus.BAD_REQUEST)); - latch.countDown(); - } - }); - latch.await(); - assertThat(searchContextCreated.get(), is(false)); - - clearScrollRequest.setScrollIds(singletonList(scrollId)); - client().clearScroll(clearScrollRequest); - } - - public void testWaitOnRefresh() throws ExecutionException, InterruptedException { - createIndex("index"); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueSeconds(30)); - searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); - - final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); - assertEquals(RestStatus.CREATED, response.status()); - - SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null, - null, - null - ); - PlainActionFuture future = new PlainActionFuture<>(); - service.executeQueryPhase(request, task, future.delegateFailure((l, r) -> { - assertEquals(1, r.queryResult().getTotalHits().value); - l.onResponse(null); - })); - future.get(); - } - - public void testWaitOnRefreshFailsWithRefreshesDisabled() { - createIndex("index", Settings.builder().put("index.refresh_interval", "-1").build()); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueSeconds(30)); - searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); - - final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); - assertEquals(RestStatus.CREATED, response.status()); - - SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); - PlainActionFuture future = new PlainActionFuture<>(); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null, - null, - null - ); - service.executeQueryPhase(request, task, future); - IllegalArgumentException illegalArgumentException = expectThrows(IllegalArgumentException.class, future::actionGet); - assertThat( - illegalArgumentException.getMessage(), - containsString("Cannot use wait_for_checkpoints with [index.refresh_interval=-1]") - ); - } - - public void testWaitOnRefreshFailsIfCheckpointNotIndexed() { - createIndex("index"); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - // Increased timeout to avoid cancelling the search task prior to its completion, - // as we expect to raise an Exception. Timeout itself is tested on the following `testWaitOnRefreshTimeout` test. - searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueMillis(randomIntBetween(200, 300))); - searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 1 })); - - final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); - assertEquals(RestStatus.CREATED, response.status()); - - SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); - PlainActionFuture future = new PlainActionFuture<>(); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null, - null, - null - ); - service.executeQueryPhase(request, task, future); - - IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, future::actionGet); - assertThat( - ex.getMessage(), - containsString("Cannot wait for unissued seqNo checkpoint [wait_for_checkpoint=1, max_issued_seqNo=0]") - ); - } - - public void testWaitOnRefreshTimeout() { - createIndex("index", Settings.builder().put("index.refresh_interval", "60s").build()); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - searchRequest.setWaitForCheckpointsTimeout(TimeValue.timeValueMillis(randomIntBetween(10, 100))); - searchRequest.setWaitForCheckpoints(Collections.singletonMap("index", new long[] { 0 })); - - final DocWriteResponse response = prepareIndex("index").setSource("id", "1").get(); - assertEquals(RestStatus.CREATED, response.status()); - - SearchShardTask task = new SearchShardTask(123L, "", "", "", null, emptyMap()); - PlainActionFuture future = new PlainActionFuture<>(); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null, - null, - null - ); - service.executeQueryPhase(request, task, future); - - ElasticsearchTimeoutException ex = expectThrows(ElasticsearchTimeoutException.class, future::actionGet); - assertThat(ex.getMessage(), containsString("Wait for seq_no [0] refreshed timed out [")); - } - - public void testMinimalSearchSourceInShardRequests() { - createIndex("test"); - int numDocs = between(0, 10); - for (int i = 0; i < numDocs; i++) { - prepareIndex("test").setSource("id", Integer.toString(i)).get(); - } - indicesAdmin().prepareRefresh("test").get(); - - BytesReference pitId = client().execute( - TransportOpenPointInTimeAction.TYPE, - new OpenPointInTimeRequest("test").keepAlive(TimeValue.timeValueMinutes(10)) - ).actionGet().getPointInTimeId(); - final MockSearchService searchService = (MockSearchService) getInstanceFromNode(SearchService.class); - final List shardRequests = new CopyOnWriteArrayList<>(); - searchService.setOnCreateSearchContext(ctx -> shardRequests.add(ctx.request())); - try { - assertHitCount( - client().prepareSearch() - .setSource( - new SearchSourceBuilder().size(between(numDocs, numDocs * 2)).pointInTimeBuilder(new PointInTimeBuilder(pitId)) - ), - numDocs - ); - } finally { - client().execute(TransportClosePointInTimeAction.TYPE, new ClosePointInTimeRequest(pitId)).actionGet(); - } - assertThat(shardRequests, not(emptyList())); - for (ShardSearchRequest shardRequest : shardRequests) { - assertNotNull(shardRequest.source()); - assertNotNull(shardRequest.source().pointInTimeBuilder()); - assertThat(shardRequest.source().pointInTimeBuilder().getEncodedId(), equalTo(BytesArray.EMPTY)); - } - } - - public void testDfsQueryPhaseRewrite() { - createIndex("index"); - prepareIndex("index").setId("1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); - final SearchService service = getInstanceFromNode(SearchService.class); - final IndicesService indicesService = getInstanceFromNode(IndicesService.class); - final IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); - final IndexShard indexShard = indexService.getShard(0); - SearchRequest searchRequest = new SearchRequest().allowPartialSearchResults(true); - searchRequest.source(SearchSourceBuilder.searchSource().query(new TestRewriteCounterQueryBuilder())); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - indexShard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1.0f, - -1, - null - ); - final Engine.SearcherSupplier reader = indexShard.acquireSearcherSupplier(); - ReaderContext context = service.createAndPutReaderContext( - request, - indexService, - indexShard, - reader, - SearchService.KEEPALIVE_INTERVAL_SETTING.get(Settings.EMPTY).millis() - ); - PlainActionFuture plainActionFuture = new PlainActionFuture<>(); - service.executeQueryPhase( - new QuerySearchRequest(null, context.id(), request, new AggregatedDfs(Map.of(), Map.of(), 10)), - new SearchShardTask(42L, "", "", "", null, emptyMap()), - plainActionFuture - ); - - plainActionFuture.actionGet(); - assertThat(((TestRewriteCounterQueryBuilder) request.source().query()).asyncRewriteCount, equalTo(1)); - final ShardSearchContextId contextId = context.id(); - assertTrue(service.freeReaderContext(contextId)); - } - - public void testEnableSearchWorkerThreads() throws IOException { - IndexService indexService = createIndex("index", Settings.EMPTY); - IndexShard indexShard = indexService.getShard(0); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - new SearchRequest().allowPartialSearchResults(randomBoolean()), - indexShard.shardId(), - 0, - indexService.numberOfShards(), - AliasFilter.EMPTY, - 1f, - System.currentTimeMillis(), - null - ); - try (ReaderContext readerContext = createReaderContext(indexService, indexShard)) { - SearchService service = getInstanceFromNode(SearchService.class); - SearchShardTask task = new SearchShardTask(0, "type", "action", "description", null, emptyMap()); - - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, randomBoolean())) { - assertNotNull(searchContext.searcher().getExecutor()); - } - - try { - ClusterUpdateSettingsResponse response = client().admin() - .cluster() - .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) - .setPersistentSettings(Settings.builder().put(SEARCH_WORKER_THREADS_ENABLED.getKey(), false).build()) - .get(); - assertTrue(response.isAcknowledged()); - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, randomBoolean())) { - assertNull(searchContext.searcher().getExecutor()); - } - } finally { - // reset original default setting - client().admin() - .cluster() - .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) - .setPersistentSettings(Settings.builder().putNull(SEARCH_WORKER_THREADS_ENABLED.getKey()).build()) - .get(); - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, randomBoolean())) { - assertNotNull(searchContext.searcher().getExecutor()); - } - } - } - } - - /** - * Verify that a single slice is created for requests that don't support parallel collection, while an executor is still - * provided to the searcher to parallelize other operations. Also ensure multiple slices are created for requests that do support - * parallel collection. - */ - public void testSlicingBehaviourForParallelCollection() throws Exception { - IndexService indexService = createIndex("index", Settings.EMPTY); - ThreadPoolExecutor executor = (ThreadPoolExecutor) indexService.getThreadPool().executor(ThreadPool.Names.SEARCH); - final int configuredMaxPoolSize = 10; - executor.setMaximumPoolSize(configuredMaxPoolSize); // We set this explicitly to be independent of CPU cores. - int numDocs = randomIntBetween(50, 100); - for (int i = 0; i < numDocs; i++) { - prepareIndex("index").setId(String.valueOf(i)).setSource("field", "value").get(); - if (i % 5 == 0) { - indicesAdmin().prepareRefresh("index").get(); - } - } - final IndexShard indexShard = indexService.getShard(0); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - new SearchRequest().allowPartialSearchResults(randomBoolean()), - indexShard.shardId(), - 0, - indexService.numberOfShards(), - AliasFilter.EMPTY, - 1f, - System.currentTimeMillis(), - null - ); - SearchService service = getInstanceFromNode(SearchService.class); - NonCountingTermQuery termQuery = new NonCountingTermQuery(new Term("field", "value")); - assertEquals(0, executor.getCompletedTaskCount()); - try (ReaderContext readerContext = createReaderContext(indexService, indexShard)) { - SearchShardTask task = new SearchShardTask(0, "type", "action", "description", null, emptyMap()); - { - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.DFS, true)) { - ContextIndexSearcher searcher = searchContext.searcher(); - assertNotNull(searcher.getExecutor()); - - final int maxPoolSize = executor.getMaximumPoolSize(); - assertEquals( - "Sanity check to ensure this isn't the default of 1 when pool size is unset", - configuredMaxPoolSize, - maxPoolSize - ); - - final int expectedSlices = ContextIndexSearcher.computeSlices( - searcher.getIndexReader().leaves(), - maxPoolSize, - 1 - ).length; - assertNotEquals("Sanity check to ensure this isn't the default of 1 when pool size is unset", 1, expectedSlices); - - final long priorExecutorTaskCount = executor.getCompletedTaskCount(); - searcher.search(termQuery, new TotalHitCountCollectorManager()); - assertBusy( - () -> assertEquals( - "DFS supports parallel collection, so the number of slices should be > 1.", - expectedSlices - 1, // one slice executes on the calling thread - executor.getCompletedTaskCount() - priorExecutorTaskCount - ) - ); - } - } - { - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.QUERY, true)) { - ContextIndexSearcher searcher = searchContext.searcher(); - assertNotNull(searcher.getExecutor()); - - final int maxPoolSize = executor.getMaximumPoolSize(); - assertEquals( - "Sanity check to ensure this isn't the default of 1 when pool size is unset", - configuredMaxPoolSize, - maxPoolSize - ); - - final int expectedSlices = ContextIndexSearcher.computeSlices( - searcher.getIndexReader().leaves(), - maxPoolSize, - 1 - ).length; - assertNotEquals("Sanity check to ensure this isn't the default of 1 when pool size is unset", 1, expectedSlices); - - final long priorExecutorTaskCount = executor.getCompletedTaskCount(); - searcher.search(termQuery, new TotalHitCountCollectorManager()); - assertBusy( - () -> assertEquals( - "QUERY supports parallel collection when enabled, so the number of slices should be > 1.", - expectedSlices - 1, // one slice executes on the calling thread - executor.getCompletedTaskCount() - priorExecutorTaskCount - ) - ); - } - } - { - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.FETCH, true)) { - ContextIndexSearcher searcher = searchContext.searcher(); - assertNull(searcher.getExecutor()); - final long priorExecutorTaskCount = executor.getCompletedTaskCount(); - searcher.search(termQuery, new TotalHitCountCollectorManager()); - assertBusy( - () -> assertEquals( - "The number of slices should be 1 as FETCH does not support parallel collection and thus runs on the calling" - + " thread.", - 0, - executor.getCompletedTaskCount() - priorExecutorTaskCount - ) - ); - } - } - { - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.NONE, true)) { - ContextIndexSearcher searcher = searchContext.searcher(); - assertNull(searcher.getExecutor()); - final long priorExecutorTaskCount = executor.getCompletedTaskCount(); - searcher.search(termQuery, new TotalHitCountCollectorManager()); - assertBusy( - () -> assertEquals( - "The number of slices should be 1 as NONE does not support parallel collection.", - 0, // zero since one slice executes on the calling thread - executor.getCompletedTaskCount() - priorExecutorTaskCount - ) - ); - } - } - - try { - ClusterUpdateSettingsResponse response = client().admin() - .cluster() - .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) - .setPersistentSettings(Settings.builder().put(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED.getKey(), false).build()) - .get(); - assertTrue(response.isAcknowledged()); - { - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.QUERY, true)) { - ContextIndexSearcher searcher = searchContext.searcher(); - assertNull(searcher.getExecutor()); - final long priorExecutorTaskCount = executor.getCompletedTaskCount(); - searcher.search(termQuery, new TotalHitCountCollectorManager()); - assertBusy( - () -> assertEquals( - "The number of slices should be 1 when QUERY parallel collection is disabled.", - 0, // zero since one slice executes on the calling thread - executor.getCompletedTaskCount() - priorExecutorTaskCount - ) - ); - } - } - } finally { - // Reset to the original default setting and check to ensure it takes effect. - client().admin() - .cluster() - .prepareUpdateSettings(TEST_REQUEST_TIMEOUT, TEST_REQUEST_TIMEOUT) - .setPersistentSettings(Settings.builder().putNull(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED.getKey()).build()) - .get(); - { - try (SearchContext searchContext = service.createContext(readerContext, request, task, ResultsType.QUERY, true)) { - ContextIndexSearcher searcher = searchContext.searcher(); - assertNotNull(searcher.getExecutor()); - - final int maxPoolSize = executor.getMaximumPoolSize(); - assertEquals( - "Sanity check to ensure this isn't the default of 1 when pool size is unset", - configuredMaxPoolSize, - maxPoolSize - ); - - final int expectedSlices = ContextIndexSearcher.computeSlices( - searcher.getIndexReader().leaves(), - maxPoolSize, - 1 - ).length; - assertNotEquals("Sanity check to ensure this isn't the default of 1 when pool size is unset", 1, expectedSlices); - - final long priorExecutorTaskCount = executor.getCompletedTaskCount(); - searcher.search(termQuery, new TotalHitCountCollectorManager()); - assertBusy( - () -> assertEquals( - "QUERY supports parallel collection when enabled, so the number of slices should be > 1.", - expectedSlices - 1, // one slice executes on the calling thread - executor.getCompletedTaskCount() - priorExecutorTaskCount - ) - ); - } - } - } - } - } - - /** - * This method tests validation that happens on the data nodes, which is now performed on the coordinating node. - * We still need the validation to cover for mixed cluster scenarios where the coordinating node does not perform the check yet. - */ - public void testParseSourceValidation() { - String index = randomAlphaOfLengthBetween(5, 10).toLowerCase(Locale.ROOT); - IndexService indexService = createIndex(index); - final SearchService service = getInstanceFromNode(SearchService.class); - { - // scroll and search_after - SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); - searchRequest.scroll(new TimeValue(1000)); - searchRequest.source().searchAfter(new String[] { "value" }); - assertCreateContextValidation(searchRequest, "`search_after` cannot be used in a scroll context.", indexService, service); - } - { - // scroll and collapse - SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); - searchRequest.scroll(new TimeValue(1000)); - searchRequest.source().collapse(new CollapseBuilder("field")); - assertCreateContextValidation(searchRequest, "cannot use `collapse` in a scroll context", indexService, service); - } - { - // search_after and `from` isn't valid - SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); - searchRequest.source().searchAfter(new String[] { "value" }); - searchRequest.source().from(10); - assertCreateContextValidation( - searchRequest, - "`from` parameter must be set to 0 when `search_after` is used", - indexService, - service - ); - } - { - // slice without scroll or pit - SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); - searchRequest.source().slice(new SliceBuilder(1, 10)); - assertCreateContextValidation( - searchRequest, - "[slice] can only be used with [scroll] or [point-in-time] requests", - indexService, - service - ); - } - { - // stored fields disabled with _source requested - SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); - searchRequest.source().storedField("_none_"); - searchRequest.source().fetchSource(true); - assertCreateContextValidation( - searchRequest, - "[stored_fields] cannot be disabled if [_source] is requested", - indexService, - service - ); - } - { - // stored fields disabled with fetch fields requested - SearchRequest searchRequest = new SearchRequest().source(new SearchSourceBuilder()); - searchRequest.source().storedField("_none_"); - searchRequest.source().fetchSource(false); - searchRequest.source().fetchField("field"); - assertCreateContextValidation( - searchRequest, - "[stored_fields] cannot be disabled when using the [fields] option", - indexService, - service - ); - } - } - - private static void assertCreateContextValidation( - SearchRequest searchRequest, - String errorMessage, - IndexService indexService, - SearchService searchService - ) { - ShardId shardId = new ShardId(indexService.index(), 0); - long nowInMillis = System.currentTimeMillis(); - String clusterAlias = randomBoolean() ? null : randomAlphaOfLengthBetween(3, 10); - searchRequest.allowPartialSearchResults(randomBoolean()); - ShardSearchRequest request = new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - shardId, - 0, - indexService.numberOfShards(), - AliasFilter.EMPTY, - 1f, - nowInMillis, - clusterAlias - ); - - SearchShardTask task = new SearchShardTask(1, "type", "action", "description", null, emptyMap()); - - ReaderContext readerContext = null; - try { - ReaderContext createOrGetReaderContext = searchService.createOrGetReaderContext(request); - readerContext = createOrGetReaderContext; - IllegalArgumentException exception = expectThrows( - IllegalArgumentException.class, - () -> searchService.createContext(createOrGetReaderContext, request, task, ResultsType.QUERY, randomBoolean()) - ); - assertThat(exception.getMessage(), containsString(errorMessage)); - } finally { - if (readerContext != null) { - readerContext.close(); - searchService.freeReaderContext(readerContext.id()); - } - } - } - - private static ReaderContext createReaderContext(IndexService indexService, IndexShard indexShard) { - return new ReaderContext( - new ShardSearchContextId(UUIDs.randomBase64UUID(), randomNonNegativeLong()), - indexService, - indexShard, - indexShard.acquireSearcherSupplier(), - randomNonNegativeLong(), - false - ); - } - - private static class TestRewriteCounterQueryBuilder extends AbstractQueryBuilder { - - final int asyncRewriteCount; - final Supplier fetched; - - TestRewriteCounterQueryBuilder() { - asyncRewriteCount = 0; - fetched = null; - } - - private TestRewriteCounterQueryBuilder(int asyncRewriteCount, Supplier fetched) { - this.asyncRewriteCount = asyncRewriteCount; - this.fetched = fetched; - } - - @Override - public String getWriteableName() { - return "test_query"; - } - - @Override - public TransportVersion getMinimalSupportedVersion() { - return TransportVersions.ZERO; - } - - @Override - protected void doWriteTo(StreamOutput out) throws IOException {} - - @Override - protected void doXContent(XContentBuilder builder, Params params) throws IOException {} - - @Override - protected Query doToQuery(SearchExecutionContext context) throws IOException { - return new MatchAllDocsQuery(); - } - - @Override - protected boolean doEquals(TestRewriteCounterQueryBuilder other) { - return true; - } - - @Override - protected int doHashCode() { - return 42; - } - - @Override - protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws IOException { - if (asyncRewriteCount > 0) { - return this; - } - if (fetched != null) { - if (fetched.get() == null) { - return this; - } - assert fetched.get(); - return new TestRewriteCounterQueryBuilder(1, null); - } - if (queryRewriteContext.convertToDataRewriteContext() != null) { - SetOnce awaitingFetch = new SetOnce<>(); - queryRewriteContext.registerAsyncAction((c, l) -> { - awaitingFetch.set(true); - l.onResponse(null); - }); - return new TestRewriteCounterQueryBuilder(0, awaitingFetch::get); - } - return this; - } } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/termsenum/action/TransportTermsEnumAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/termsenum/action/TransportTermsEnumAction.java index 9164fd88b6395..08e89a0fcab00 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/termsenum/action/TransportTermsEnumAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/termsenum/action/TransportTermsEnumAction.java @@ -492,7 +492,7 @@ private boolean hasMatchAllEquivalent( return false; } - private boolean canMatchShard(ShardId shardId, NodeTermsEnumRequest req) throws IOException { + private boolean canMatchShard(ShardId shardId, NodeTermsEnumRequest req) { if (req.indexFilter() == null || req.indexFilter() instanceof MatchAllQueryBuilder) { return true; } diff --git a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java index 8ba88865e361a..89d80cf34aec5 100644 --- a/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java +++ b/x-pack/plugin/frozen-indices/src/internalClusterTest/java/org/elasticsearch/index/engine/frozen/FrozenIndexTests.java @@ -46,7 +46,6 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.protocol.xpack.frozen.FreezeRequest; import org.elasticsearch.rest.RestStatus; -import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -469,27 +468,25 @@ public void testCanMatch() throws IOException { ).canMatch() ); - expectThrows(SearchContextMissingException.class, () -> { - ShardSearchContextId withoutCommitId = new ShardSearchContextId(contextId.getSessionId(), contextId.getId(), null); - sourceBuilder.query(QueryBuilders.rangeQuery("field").gt("2010-01-06T02:00").lt("2010-01-07T02:00")); - assertFalse( - searchService.canMatch( - new ShardSearchRequest( - OriginalIndices.NONE, - searchRequest, - shard.shardId(), - 0, - 1, - AliasFilter.EMPTY, - 1f, - -1, - null, - withoutCommitId, - null - ) - ).canMatch() - ); - }); + ShardSearchContextId withoutCommitId = new ShardSearchContextId(contextId.getSessionId(), contextId.getId(), null); + sourceBuilder.query(QueryBuilders.rangeQuery("field").gt("2010-01-06T02:00").lt("2010-01-07T02:00")); + assertTrue( + searchService.canMatch( + new ShardSearchRequest( + OriginalIndices.NONE, + searchRequest, + shard.shardId(), + 0, + 1, + AliasFilter.EMPTY, + 1f, + -1, + null, + withoutCommitId, + null + ) + ).canMatch() + ); } } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java index 3ee49cce85a8a..6115bec91ad62 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/BaseSearchableSnapshotsIntegTestCase.java @@ -355,6 +355,18 @@ protected void assertExecutorIsIdle(String executorName) throws Exception { }); } + protected static void waitUntilRecoveryIsDone(String index) throws Exception { + assertBusy(() -> { + RecoveryResponse recoveryResponse = indicesAdmin().prepareRecoveries(index).get(); + assertThat(recoveryResponse.hasRecoveries(), equalTo(true)); + for (List value : recoveryResponse.shardRecoveryStates().values()) { + for (RecoveryState recoveryState : value) { + assertThat(recoveryState.getStage(), equalTo(RecoveryState.Stage.DONE)); + } + } + }); + } + public static class LicensedSnapshotBasedRecoveriesPlugin extends SnapshotBasedRecoveriesPlugin { public LicensedSnapshotBasedRecoveriesPlugin(Settings settings) { diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java index 69c5141fcef41..c9dcac631ea13 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsCanMatchOnCoordinatorIntegTests.java @@ -7,7 +7,6 @@ package org.elasticsearch.xpack.searchablesnapshots; -import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchRequest; @@ -36,7 +35,6 @@ import org.elasticsearch.index.shard.IndexLongFieldRange; import org.elasticsearch.indices.DateFieldRangeInfo; import org.elasticsearch.indices.IndicesService; -import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.snapshots.SnapshotId; @@ -1334,18 +1332,6 @@ private static IndexMetadata getIndexMetadata(String indexName) { .index(indexName); } - private static void waitUntilRecoveryIsDone(String index) throws Exception { - assertBusy(() -> { - RecoveryResponse recoveryResponse = indicesAdmin().prepareRecoveries(index).get(); - assertThat(recoveryResponse.hasRecoveries(), equalTo(true)); - for (List value : recoveryResponse.shardRecoveryStates().values()) { - for (RecoveryState recoveryState : value) { - assertThat(recoveryState.getStage(), equalTo(RecoveryState.Stage.DONE)); - } - } - }); - } - private void waitUntilAllShardsAreUnassigned(Index index) throws Exception { awaitClusterState(state -> state.getRoutingTable().index(index).allPrimaryShardsUnassigned()); } diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSearchIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSearchIntegTests.java new file mode 100644 index 0000000000000..c60ebc884dbed --- /dev/null +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsSearchIntegTests.java @@ -0,0 +1,129 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License + * 2.0; you may not use this file except in compliance with the Elastic License + * 2.0. + */ + +package org.elasticsearch.xpack.searchablesnapshots; + +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchRequest; +import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.search.builder.SearchSourceBuilder; +import org.elasticsearch.snapshots.SnapshotId; +import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotAction; +import org.elasticsearch.xpack.core.searchablesnapshots.MountSearchableSnapshotRequest; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; + +import static org.elasticsearch.cluster.metadata.IndexMetadata.INDEX_ROUTING_REQUIRE_GROUP_SETTING; +import static org.elasticsearch.index.IndexSettings.INDEX_SOFT_DELETES_SETTING; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertResponse; +import static org.hamcrest.Matchers.equalTo; + +public class SearchableSnapshotsSearchIntegTests extends BaseFrozenSearchableSnapshotsIntegTestCase { + + /** + * Tests basic search functionality with a query sorted by field against partially mounted indices + * The can match phase is always executed against read only indices, and for sorted queries it extracts the min and max range from + * each shard. This will happen not only in the can match phase, but optionally also in the query phase. + * See {@link org.elasticsearch.search.internal.ShardSearchRequest#canReturnNullResponseIfMatchNoDocs()}. + * For keyword fields, it is not possible to retrieve min and max from the index reader on frozen, hence we need to make sure that + * while that fails, the query will go ahead and won't return shard failures. + */ + public void testKeywordSortedQueryOnFrozen() throws Exception { + internalCluster().startMasterOnlyNode(); + internalCluster().startCoordinatingOnlyNode(Settings.EMPTY); + final String dataNodeHoldingRegularIndex = internalCluster().startDataOnlyNode(); + String dataNodeHoldingSearchableSnapshot = internalCluster().startDataOnlyNode(); + + String[] indices = new String[] { "index-0001", "index-0002" }; + for (String index : indices) { + Settings extraSettings = Settings.builder() + .put(INDEX_ROUTING_REQUIRE_GROUP_SETTING.getConcreteSettingForNamespace("_name").getKey(), dataNodeHoldingRegularIndex) + .build(); + // we use a high number of shards because that's more likely to trigger can match as part of query phase: + // see ShardSearchRequest#canReturnNullResponseIfMatchNoDocs + assertAcked( + indicesAdmin().prepareCreate(index) + .setSettings(indexSettingsNoReplicas(10).put(INDEX_SOFT_DELETES_SETTING.getKey(), true).put(extraSettings)) + ); + } + ensureGreen(indices); + + for (String index : indices) { + final List indexRequestBuilders = new ArrayList<>(); + indexRequestBuilders.add(prepareIndex(index).setSource("keyword", "value1")); + indexRequestBuilders.add(prepareIndex(index).setSource("keyword", "value2")); + indexRandom(true, false, indexRequestBuilders); + assertThat( + indicesAdmin().prepareForceMerge(index).setOnlyExpungeDeletes(true).setFlush(true).get().getFailedShards(), + equalTo(0) + ); + refresh(index); + forceMerge(); + } + + final String repositoryName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createRepository(repositoryName, "mock"); + + final SnapshotId snapshotId = createSnapshot(repositoryName, "snapshot-1", List.of(indices[0])).snapshotId(); + assertAcked(indicesAdmin().prepareDelete(indices[0])); + + // Block the repository for the node holding the searchable snapshot shards + // to delay its restore + blockDataNode(repositoryName, dataNodeHoldingSearchableSnapshot); + + // Force the searchable snapshot to be allocated in a particular node + Settings restoredIndexSettings = Settings.builder() + .put(INDEX_ROUTING_REQUIRE_GROUP_SETTING.getConcreteSettingForNamespace("_name").getKey(), dataNodeHoldingSearchableSnapshot) + .build(); + String[] mountedIndices = new String[indices.length]; + for (int i = 0; i < indices.length; i++) { + + String index = indices[i]; + String mountedIndex = index + "-mounted"; + mountedIndices[i] = mountedIndex; + final MountSearchableSnapshotRequest mountRequest = new MountSearchableSnapshotRequest( + TEST_REQUEST_TIMEOUT, + mountedIndex, + repositoryName, + snapshotId.getName(), + indices[0], + restoredIndexSettings, + Strings.EMPTY_ARRAY, + false, + randomFrom(MountSearchableSnapshotRequest.Storage.values()) + ); + client().execute(MountSearchableSnapshotAction.INSTANCE, mountRequest).actionGet(); + } + + // Allow the searchable snapshots to be finally mounted + unblockNode(repositoryName, dataNodeHoldingSearchableSnapshot); + for (String mountedIndex : mountedIndices) { + waitUntilRecoveryIsDone(mountedIndex); + } + ensureGreen(mountedIndices); + + SearchRequest request = new SearchRequest(mountedIndices).searchType(SearchType.QUERY_THEN_FETCH) + .source(SearchSourceBuilder.searchSource().sort("keyword.keyword")) + .allowPartialSearchResults(false); + if (randomBoolean()) { + request.setPreFilterShardSize(100); + } + + assertResponse(client().search(request), searchResponse -> { + assertThat(searchResponse.getSuccessfulShards(), equalTo(20)); + assertThat(searchResponse.getFailedShards(), equalTo(0)); + assertThat(searchResponse.getSkippedShards(), equalTo(0)); + assertThat(searchResponse.getTotalShards(), equalTo(20)); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(4L)); + }); + } +} From a33cc0fc1cc819c55a4c16f0c83a8dd3cdb7a743 Mon Sep 17 00:00:00 2001 From: elasticsearchmachine <58790826+elasticsearchmachine@users.noreply.github.com> Date: Fri, 13 Dec 2024 03:00:09 +1100 Subject: [PATCH 2/6] Remove 8.15 from branches.json --- branches.json | 3 --- 1 file changed, 3 deletions(-) diff --git a/branches.json b/branches.json index 0e23a795664dd..95fbdb1efd655 100644 --- a/branches.json +++ b/branches.json @@ -13,9 +13,6 @@ { "branch": "8.x" }, - { - "branch": "8.15" - }, { "branch": "7.17" } From c2208698247082553ecaa79520fca477ff45d522 Mon Sep 17 00:00:00 2001 From: Mark Vieira Date: Thu, 12 Dec 2024 11:28:16 -0800 Subject: [PATCH 3/6] Remove version 8.15.6 --- .buildkite/pipelines/intake.yml | 2 +- .buildkite/pipelines/periodic-packaging.yml | 6 +++--- .buildkite/pipelines/periodic.yml | 10 +++++----- .ci/bwcVersions | 2 +- .ci/snapshotBwcVersions | 1 - server/src/main/java/org/elasticsearch/Version.java | 1 - 6 files changed, 10 insertions(+), 12 deletions(-) diff --git a/.buildkite/pipelines/intake.yml b/.buildkite/pipelines/intake.yml index 12ea14eed1cd3..cb527ae0c2085 100644 --- a/.buildkite/pipelines/intake.yml +++ b/.buildkite/pipelines/intake.yml @@ -56,7 +56,7 @@ steps: timeout_in_minutes: 300 matrix: setup: - BWC_VERSION: ["7.17.27", "8.15.6", "8.16.2"] + BWC_VERSION: ["7.17.27", "8.16.2"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.buildkite/pipelines/periodic-packaging.yml b/.buildkite/pipelines/periodic-packaging.yml index 9772e465d4a5f..6329d94cda67f 100644 --- a/.buildkite/pipelines/periodic-packaging.yml +++ b/.buildkite/pipelines/periodic-packaging.yml @@ -559,8 +559,8 @@ steps: env: BWC_VERSION: 8.14.3 - - label: "{{matrix.image}} / 8.15.6 / packaging-tests-upgrade" - command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.15.6 + - label: "{{matrix.image}} / 8.15.5 / packaging-tests-upgrade" + command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.15.5 timeout_in_minutes: 300 matrix: setup: @@ -573,7 +573,7 @@ steps: machineType: custom-16-32768 buildDirectory: /dev/shm/bk env: - BWC_VERSION: 8.15.6 + BWC_VERSION: 8.15.5 - label: "{{matrix.image}} / 8.16.2 / packaging-tests-upgrade" command: ./.ci/scripts/packaging-test.sh -Dbwc.checkout.align=true destructiveDistroUpgradeTest.v8.16.2 diff --git a/.buildkite/pipelines/periodic.yml b/.buildkite/pipelines/periodic.yml index 002404e0b8198..dfefe5dc99642 100644 --- a/.buildkite/pipelines/periodic.yml +++ b/.buildkite/pipelines/periodic.yml @@ -629,8 +629,8 @@ steps: - signal_reason: agent_stop limit: 3 - - label: 8.15.6 / bwc - command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v8.15.6#bwcTest + - label: 8.15.5 / bwc + command: .ci/scripts/run-gradle.sh -Dbwc.checkout.align=true v8.15.5#bwcTest timeout_in_minutes: 300 agents: provider: gcp @@ -639,7 +639,7 @@ steps: buildDirectory: /dev/shm/bk preemptible: true env: - BWC_VERSION: 8.15.6 + BWC_VERSION: 8.15.5 retry: automatic: - exit_status: "-1" @@ -733,7 +733,7 @@ steps: setup: ES_RUNTIME_JAVA: - openjdk17 - BWC_VERSION: ["7.17.27", "8.15.6", "8.16.2"] + BWC_VERSION: ["7.17.27", "8.16.2"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 @@ -781,7 +781,7 @@ steps: - openjdk21 - openjdk22 - openjdk23 - BWC_VERSION: ["7.17.27", "8.15.6", "8.16.2"] + BWC_VERSION: ["7.17.27", "8.16.2"] agents: provider: gcp image: family/elasticsearch-ubuntu-2004 diff --git a/.ci/bwcVersions b/.ci/bwcVersions index 8d6d2779fdd46..dfcae800a14e0 100644 --- a/.ci/bwcVersions +++ b/.ci/bwcVersions @@ -32,5 +32,5 @@ BWC_VERSION: - "8.12.2" - "8.13.4" - "8.14.3" - - "8.15.6" + - "8.15.5" - "8.16.2" diff --git a/.ci/snapshotBwcVersions b/.ci/snapshotBwcVersions index 8d4c53e62657d..322e2b855f02a 100644 --- a/.ci/snapshotBwcVersions +++ b/.ci/snapshotBwcVersions @@ -1,4 +1,3 @@ BWC_VERSION: - "7.17.27" - - "8.15.6" - "8.16.2" diff --git a/server/src/main/java/org/elasticsearch/Version.java b/server/src/main/java/org/elasticsearch/Version.java index f8abeb8e30488..100420f9184c3 100644 --- a/server/src/main/java/org/elasticsearch/Version.java +++ b/server/src/main/java/org/elasticsearch/Version.java @@ -190,7 +190,6 @@ public class Version implements VersionId, ToXContentFragment { public static final Version V_8_15_3 = new Version(8_15_03_99); public static final Version V_8_15_4 = new Version(8_15_04_99); public static final Version V_8_15_5 = new Version(8_15_05_99); - public static final Version V_8_15_6 = new Version(8_15_06_99); public static final Version V_8_16_0 = new Version(8_16_00_99); public static final Version V_8_16_1 = new Version(8_16_01_99); public static final Version V_8_16_2 = new Version(8_16_02_99); From a4291cc1aa83a8495c29068b66817f63e5c7c634 Mon Sep 17 00:00:00 2001 From: Nik Everett Date: Thu, 12 Dec 2024 17:17:01 -0500 Subject: [PATCH 4/6] Remove ESQL INLINESTATS from changelog (#118627) ESQL's INLINESTATS is still experimental and not being released with 8.16. I was too excited when I wrote it and marked it as a feature. But it's not released, so not really a feature. --- docs/reference/release-notes/8.16.0.asciidoc | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/reference/release-notes/8.16.0.asciidoc b/docs/reference/release-notes/8.16.0.asciidoc index 88ae9f9e5b599..fd7ef963d9ff7 100644 --- a/docs/reference/release-notes/8.16.0.asciidoc +++ b/docs/reference/release-notes/8.16.0.asciidoc @@ -270,7 +270,6 @@ ES|QL:: * Push down filters even in case of renames in Evals {es-pull}114411[#114411] * Speed up CASE for some parameters {es-pull}112295[#112295] * Speed up grouping by bytes {es-pull}114021[#114021] -* Support INLINESTATS grouped on expressions {es-pull}111690[#111690] * Use less memory in listener {es-pull}114358[#114358] * Add support for cached strings in plan serialization {es-pull}112929[#112929] * Add Telemetry API and track top functions {es-pull}111226[#111226] @@ -462,7 +461,6 @@ ES|QL:: * Add boolean support to Max and Min aggs {es-pull}110527[#110527] * Add boolean support to TOP aggregation {es-pull}110718[#110718] * Added `mv_percentile` function {es-pull}111749[#111749] (issue: {es-issue}111591[#111591]) -* INLINESTATS {es-pull}109583[#109583] (issue: {es-issue}107589[#107589]) * Introduce per agg filter {es-pull}113735[#113735] * Strings support for MAX and MIN aggregations {es-pull}111544[#111544] * Support IP fields in MAX and MIN aggregations {es-pull}110921[#110921] From cf7fb3ed8320da8c62a744da9e95d7861d86158c Mon Sep 17 00:00:00 2001 From: Rene Groeschke Date: Fri, 13 Dec 2024 18:46:08 +0100 Subject: [PATCH 5/6] [8.16] Add wolfi changelog (#118684) * Add wolfi image as enhancement to changelogs * Tweak yaml format * Update docs regarding docker * Update changelog after details is not permitted and push generated release notes * Delete docs/changelog/118684.yaml * Update docs/changelog/118684.yaml * Bring back changelog file --- docs/Versions.asciidoc | 1 + docs/changelog/118684.yaml | 8 +++ docs/reference/release-notes.asciidoc | 2 + docs/reference/release-notes/8.16.2.asciidoc | 65 ++++++++++++++++++++ docs/reference/setup/install/docker.asciidoc | 6 ++ 5 files changed, 82 insertions(+) create mode 100644 docs/changelog/118684.yaml create mode 100644 docs/reference/release-notes/8.16.2.asciidoc diff --git a/docs/Versions.asciidoc b/docs/Versions.asciidoc index b65b974cd6b69..e19db10783988 100644 --- a/docs/Versions.asciidoc +++ b/docs/Versions.asciidoc @@ -9,6 +9,7 @@ include::{docs-root}/shared/versions/stack/{source_branch}.asciidoc[] :docker-repo: docker.elastic.co/elasticsearch/elasticsearch :docker-image: {docker-repo}:{version} +:docker-wolfi-image: {docker-repo}-wolfi:{version} :kib-docker-repo: docker.elastic.co/kibana/kibana :kib-docker-image: {kib-docker-repo}:{version} :plugin_url: https://artifacts.elastic.co/downloads/elasticsearch-plugins diff --git a/docs/changelog/118684.yaml b/docs/changelog/118684.yaml new file mode 100644 index 0000000000000..c550f92dcd432 --- /dev/null +++ b/docs/changelog/118684.yaml @@ -0,0 +1,8 @@ +pr: 118684 +summary: |- + In this release we've introduced an image based on the hardened link:https://github.com/wolfi-dev/[Wolfi] + image to provide secure containers to our self-managed customers, help with compliance regulations, + and improve our supply chain security posture. +area: Packaging +type: enhancement +issues: [] diff --git a/docs/reference/release-notes.asciidoc b/docs/reference/release-notes.asciidoc index fe4221bab6cb2..4f6dad223ecea 100644 --- a/docs/reference/release-notes.asciidoc +++ b/docs/reference/release-notes.asciidoc @@ -6,6 +6,7 @@ This section summarizes the changes in each release. +* <> * <> * <> * <> @@ -77,6 +78,7 @@ This section summarizes the changes in each release. -- +include::release-notes/8.16.2.asciidoc[] include::release-notes/8.16.1.asciidoc[] include::release-notes/8.16.0.asciidoc[] include::release-notes/8.15.5.asciidoc[] diff --git a/docs/reference/release-notes/8.16.2.asciidoc b/docs/reference/release-notes/8.16.2.asciidoc new file mode 100644 index 0000000000000..9e981ffe5835d --- /dev/null +++ b/docs/reference/release-notes/8.16.2.asciidoc @@ -0,0 +1,65 @@ +[[release-notes-8.16.2]] +== {es} version 8.16.2 + +coming[8.16.2] + +Also see <>. + +[[bug-8.16.2]] +[float] +=== Bug fixes + +CCS:: +* Fix long metric deserialize & add - auto-resize needs to be set manually {es-pull}117105[#117105] (issue: {es-issue}116914[#116914]) + +Data streams:: +* Acquire stats searcher for data stream stats {es-pull}117953[#117953] + +ES|QL:: +* ESQL: fix the column position in errors {es-pull}117153[#117153] +* Fix BWC for ES|QL cluster request {es-pull}117865[#117865] +* Fix CCS exchange when multi cluster aliases point to same cluster {es-pull}117297[#117297] +* Fix COUNT filter pushdown {es-pull}117503[#117503] (issue: {es-issue}115522[#115522]) +* Fix validation of SORT by aggregate functions {es-pull}117316[#117316] +* Opt into extra data stream resolution {es-pull}118378[#118378] + +Ingest Node:: +* Fix enrich cache size setting name {es-pull}117575[#117575] +* Fix log message format bugs {es-pull}118354[#118354] +* Fix reconstituting version string from components {es-pull}117213[#117213] (issue: {es-issue}116950[#116950]) + +License:: +* Distinguish `LicensedFeature` by family field {es-pull}116809[#116809] + +Logs:: +* Prohibit changes to index mode, source, and sort settings during resize {es-pull}115812[#115812] + +Machine Learning:: +* Fix deberta tokenizer bug caused by bug in normalizer {es-pull}117189[#117189] +* Fix for Deberta tokenizer when input sequence exceeds 512 tokens {es-pull}117595[#117595] +* Propagate scoring function through random sampler {es-pull}116957[#116957] (issue: {es-issue}110134[#110134]) +* Update Deberta tokenizer {es-pull}116358[#116358] + +Mapping:: +* Restore original "is within leaf" value in `SparseVectorFieldMapper` {es-pull}118380[#118380] + +Ranking:: +* Fix for propagating filters from compound to inner retrievers {es-pull}117914[#117914] + +Search:: +* Don't skip shards in coord rewrite if timestamp is an alias {es-pull}117271[#117271] +* Handle exceptions in query phase can match {es-pull}117469[#117469] (issue: {es-issue}104994[#104994]) + +[[enhancement-8.16.2]] +[float] +=== Enhancements + +Inference:: +* [8.16] Update sparse text embeddings API route for Inference Service {es-pull}118367[#118367] + +Packaging:: +* In this release we've introduced an image based on the hardened link:https://github.com/wolfi-dev/[Wolfi] +image to provide secure containers to our self-managed customers, help with compliance regulations, +and improve our supply chain security posture. {es-pull}118684[#118684] + + diff --git a/docs/reference/setup/install/docker.asciidoc b/docs/reference/setup/install/docker.asciidoc index 8694d7f5b46c6..86a0e567f6eec 100644 --- a/docs/reference/setup/install/docker.asciidoc +++ b/docs/reference/setup/install/docker.asciidoc @@ -55,6 +55,12 @@ docker pull {docker-image} // REVIEWED[DEC.10.24] -- +Alternatevely, you can use the Wolfi based image. Using Wolfi based images requires Docker version 20.10.10 or superior. +[source,sh,subs="attributes"] +---- +docker pull {docker-wolfi-image} +---- + . Optional: Install https://docs.sigstore.dev/cosign/system_config/installation/[Cosign] for your environment. Then use Cosign to verify the {es} image's signature. From 825b4763ce7e0b6ba0a411209dd1fe103fa6e6b9 Mon Sep 17 00:00:00 2001 From: Rene Groeschke Date: Sun, 15 Dec 2024 02:30:47 +0100 Subject: [PATCH 6/6] Update docker.elastic.co/wolfi/chainguard-base:latest Docker digest to 1b51ff6 (#117902) (#118724) Co-authored-by: elastic-renovate-prod[bot] <174716857+elastic-renovate-prod[bot]@users.noreply.github.com> Co-authored-by: Rene Groeschke (cherry picked from commit 21b7afd6692105fc8f79b9c02ef4999c6e9a0b18) --- .../java/org/elasticsearch/gradle/internal/DockerBase.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java index fa3f6c3b7400b..b86ce97c182fe 100644 --- a/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java +++ b/build-tools-internal/src/main/java/org/elasticsearch/gradle/internal/DockerBase.java @@ -27,8 +27,7 @@ public enum DockerBase { // Chainguard based wolfi image with latest jdk // This is usually updated via renovatebot // spotless:off - WOLFI( - "docker.elastic.co/wolfi/chainguard-base:latest@sha256:90888b190da54062f67f3fef1372eb0ae7d81ea55f5a1f56d748b13e4853d984", + WOLFI("docker.elastic.co/wolfi/chainguard-base:latest@sha256:1b51ff6dba78c98d3e02b0cd64a8ce3238c7a40408d21e3af12a329d44db6f23", "-wolfi", "apk" ),