From 5a98b8d9dfb4c93b17cd849b14edf7e67c28d53a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 7 Nov 2024 14:34:52 +0100 Subject: [PATCH 001/132] Make SearchResponseSections Releasable instead of RefCounted This is not used as ref-counted, we never increment the count so we can simplify things a little and make it just a releasable. --- .../action/search/FetchSearchPhase.java | 2 +- .../action/search/SearchResponseSections.java | 32 +++---------------- .../search/SearchScrollAsyncAction.java | 5 +-- .../action/search/ExpandSearchPhaseTests.java | 22 ++++--------- .../search/FetchLookupFieldsPhaseTests.java | 8 ++--- .../search/SearchPhaseControllerTests.java | 8 ++--- 6 files changed, 16 insertions(+), 61 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index d7b847d835b83..b5a3badde5fb8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -271,7 +271,7 @@ private void moveToNextPhase( ) { context.executeNextPhase(this, () -> { var resp = SearchPhaseController.merge(context.getRequest().scroll() != null, reducedQueryPhase, fetchResultsArr); - context.addReleasable(resp::decRef); + context.addReleasable(resp); return nextPhaseFactory.apply(resp, searchPhaseShardResults); }); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchResponseSections.java b/server/src/main/java/org/elasticsearch/action/search/SearchResponseSections.java index 8c9a42a61e33e..9d85348b80d62 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchResponseSections.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchResponseSections.java @@ -9,14 +9,12 @@ package org.elasticsearch.action.search; -import org.elasticsearch.core.RefCounted; -import org.elasticsearch.core.SimpleRefCounted; +import org.elasticsearch.core.Releasable; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.profile.SearchProfileResults; import org.elasticsearch.search.profile.SearchProfileShardResult; import org.elasticsearch.search.suggest.Suggest; -import org.elasticsearch.transport.LeakTracker; import java.util.Collections; import java.util.Map; @@ -25,7 +23,7 @@ * Holds some sections that a search response is composed of (hits, aggs, suggestions etc.) during some steps of the search response * building. */ -public class SearchResponseSections implements RefCounted { +public class SearchResponseSections implements Releasable { public static final SearchResponseSections EMPTY_WITH_TOTAL_HITS = new SearchResponseSections( SearchHits.EMPTY_WITH_TOTAL_HITS, @@ -53,8 +51,6 @@ public class SearchResponseSections implements RefCounted { protected final Boolean terminatedEarly; protected final int numReducePhases; - private final RefCounted refCounted; - public SearchResponseSections( SearchHits hits, InternalAggregations aggregations, @@ -72,7 +68,6 @@ public SearchResponseSections( this.timedOut = timedOut; this.terminatedEarly = terminatedEarly; this.numReducePhases = numReducePhases; - refCounted = hits.getHits().length > 0 ? LeakTracker.wrap(new SimpleRefCounted()) : ALWAYS_REFERENCED; } public final SearchHits hits() { @@ -97,26 +92,7 @@ public final Map profile() { } @Override - public void incRef() { - refCounted.incRef(); - } - - @Override - public boolean tryIncRef() { - return refCounted.tryIncRef(); - } - - @Override - public boolean decRef() { - if (refCounted.decRef()) { - hits.decRef(); - return true; - } - return false; - } - - @Override - public boolean hasReferences() { - return refCounted.hasReferences(); + public void close() { + hits.decRef(); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java index 60e96a8cce8ab..2231f791384fa 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchScrollAsyncAction.java @@ -246,8 +246,7 @@ protected final void sendResponse( if (request.scroll() != null) { scrollId = request.scrollId(); } - var sections = SearchPhaseController.merge(true, queryPhase, fetchResults); - try { + try (var sections = SearchPhaseController.merge(true, queryPhase, fetchResults)) { ActionListener.respondAndRelease( listener, new SearchResponse( @@ -262,8 +261,6 @@ protected final void sendResponse( null ) ); - } finally { - sections.decRef(); } } catch (Exception e) { listener.onFailure(new ReduceSearchPhaseException("fetch", "inner finish failed", e, buildShardFailures())); diff --git a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java index 5240d704dea3b..70c7c07a9df25 100644 --- a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java @@ -96,11 +96,10 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL List mSearchResponses = new ArrayList<>(numInnerHits); for (int innerHitNum = 0; innerHitNum < numInnerHits; innerHitNum++) { - var sections = new SearchResponseSections(collapsedHits.get(innerHitNum), null, null, false, null, null, 1); - try { + try ( + var sections = new SearchResponseSections(collapsedHits.get(innerHitNum), null, null, false, null, null, 1) + ) { mockSearchPhaseContext.sendSearchResponse(sections, null); - } finally { - sections.decRef(); } mSearchResponses.add(new MultiSearchResponse.Item(mockSearchPhaseContext.searchResponse.get(), null)); // transferring ownership to the multi-search response so no need to release here @@ -121,11 +120,8 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") { @Override public void run() { - var sections = new SearchResponseSections(hits, null, null, false, null, null, 1); - try { + try (var sections = new SearchResponseSections(hits, null, null, false, null, null, 1)) { mockSearchPhaseContext.sendSearchResponse(sections, null); - } finally { - sections.decRef(); } } }); @@ -215,11 +211,8 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") { @Override public void run() { - var sections = new SearchResponseSections(hits, null, null, false, null, null, 1); - try { + try (var sections = new SearchResponseSections(hits, null, null, false, null, null, 1)) { mockSearchPhaseContext.sendSearchResponse(sections, null); - } finally { - sections.decRef(); } } }); @@ -254,11 +247,8 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL ExpandSearchPhase phase = new ExpandSearchPhase(mockSearchPhaseContext, hits, () -> new SearchPhase("test") { @Override public void run() { - var sections = new SearchResponseSections(hits, null, null, false, null, null, 1); - try { + try (var sections = new SearchResponseSections(hits, null, null, false, null, null, 1)) { mockSearchPhaseContext.sendSearchResponse(sections, null); - } finally { - sections.decRef(); } } }); diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java index e478ed2d0ccb5..baabed0ad4c3e 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchLookupFieldsPhaseTests.java @@ -47,12 +47,10 @@ void sendExecuteMultiSearch(MultiSearchRequest request, SearchTask task, ActionL searchHits[i] = SearchHitTests.createTestItem(randomBoolean(), randomBoolean()); } SearchHits hits = new SearchHits(searchHits, new TotalHits(numHits, TotalHits.Relation.EQUAL_TO), 1.0f); - var sections = new SearchResponseSections(hits, null, null, false, null, null, 1); - try { + try (var sections = new SearchResponseSections(hits, null, null, false, null, null, 1)) { FetchLookupFieldsPhase phase = new FetchLookupFieldsPhase(searchPhaseContext, sections, null); phase.run(); } finally { - sections.decRef(); hits.decRef(); } searchPhaseContext.assertNoFailure(); @@ -190,12 +188,10 @@ void sendExecuteMultiSearch( new TotalHits(2, TotalHits.Relation.EQUAL_TO), 1.0f ); - var sections = new SearchResponseSections(searchHits, null, null, false, null, null, 1); - try { + try (var sections = new SearchResponseSections(searchHits, null, null, false, null, null, 1)) { FetchLookupFieldsPhase phase = new FetchLookupFieldsPhase(searchPhaseContext, sections, null); phase.run(); } finally { - sections.decRef(); searchHits.decRef(); } assertTrue(requestSent.get()); diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index 9a507977c0123..bf81486087361 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -292,8 +292,7 @@ public void testMerge() { reducedQueryPhase.suggest(), profile ); - final SearchResponseSections mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults); - try { + try (SearchResponseSections mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults)) { if (trackTotalHits == SearchContext.TRACK_TOTAL_HITS_DISABLED) { assertNull(mergedResponse.hits.getTotalHits()); } else { @@ -346,7 +345,6 @@ public void testMerge() { assertThat(mergedResponse.profile(), is(anEmptyMap())); } } finally { - mergedResponse.decRef(); fetchResults.asList().forEach(TransportMessage::decRef); } } finally { @@ -410,8 +408,7 @@ protected boolean lessThan(RankDoc a, RankDoc b) { reducedQueryPhase.suggest(), false ); - SearchResponseSections mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults); - try { + try (SearchResponseSections mergedResponse = SearchPhaseController.merge(false, reducedQueryPhase, fetchResults)) { if (trackTotalHits == SearchContext.TRACK_TOTAL_HITS_DISABLED) { assertNull(mergedResponse.hits.getTotalHits()); } else { @@ -427,7 +424,6 @@ protected boolean lessThan(RankDoc a, RankDoc b) { assertThat(mergedResponse.hits().getHits().length, equalTo(reducedQueryPhase.sortedTopDocs().scoreDocs().length)); assertThat(mergedResponse.profile(), is(anEmptyMap())); } finally { - mergedResponse.decRef(); fetchResults.asList().forEach(TransportMessage::decRef); } } finally { From f078e4a1afbf3eb02d17ba8220ca7794e92e994f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 15 Nov 2024 11:55:46 +0100 Subject: [PATCH 002/132] Faster+Simpler CanMatchPreFilterSearchPhase This redose CanMatchPreFilterSearchPhase to be simpler and without the needless heavy locking. We were locking on every shard result which could introduce a lot of contention when a large number of data nodes (especially when they hold large shard counts) are queried. We probably could even do without the volatile in some cases but either way, this commit does away with any contention on the results since we will never access the same cell in the result array from more than a single thread. There is no need to have a bitset here and deal with the synchronization it entails when we at the same time also set up two arrays of shard_count length and only need a single array to begin with. => share array for failures, min-max and matches, remove all synchronization, further simplify the code. --- .../search/CanMatchPreFilterSearchPhase.java | 400 ++++++++---------- .../TransportOpenPointInTimeAction.java | 4 +- .../action/search/TransportSearchAction.java | 12 +- .../search/TransportSearchShardsAction.java | 4 +- .../search/CanMatchShardResponse.java | 4 +- .../CanMatchPreFilterSearchPhaseTests.java | 31 +- 6 files changed, 195 insertions(+), 260 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index eaf62d1e57e66..b28bf6457b970 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -10,7 +10,6 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; -import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.util.Maps; @@ -41,8 +40,6 @@ import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; -import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.elasticsearch.core.Strings.format; import static org.elasticsearch.core.Types.forciblyCast; @@ -60,6 +57,9 @@ */ final class CanMatchPreFilterSearchPhase { + /** Value to use in {@link #res} for shards that match but where no {@link MinAndMax} value is available*/ + private static final Object TRUE_SENTINEL = new Object(); + private final Logger logger; private final SearchRequest request; private final GroupShardsIterator shardsIts; @@ -74,10 +74,11 @@ final class CanMatchPreFilterSearchPhase { private final Executor executor; private final boolean requireAtLeastOneMatch; - private final CanMatchSearchPhaseResults results; private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; - CanMatchPreFilterSearchPhase( + private final AtomicReferenceArray res; + + private CanMatchPreFilterSearchPhase( Logger logger, SearchTransportService searchTransportService, BiFunction nodeIdToConnection, @@ -105,12 +106,13 @@ final class CanMatchPreFilterSearchPhase { this.requireAtLeastOneMatch = requireAtLeastOneMatch; this.coordinatorRewriteContextProvider = coordinatorRewriteContextProvider; this.executor = executor; - results = new CanMatchSearchPhaseResults(shardsIts.size()); + int size = shardsIts.size(); + res = new AtomicReferenceArray<>(size); // we compute the shard index based on the natural order of the shards // that participate in the search request. This means that this number is // consistent between two requests that target the same shards. - final SearchShardIterator[] naturalOrder = new SearchShardIterator[shardsIts.size()]; + final SearchShardIterator[] naturalOrder = new SearchShardIterator[size]; int i = 0; for (SearchShardIterator shardsIt : shardsIts) { naturalOrder[i++] = shardsIt; @@ -123,6 +125,42 @@ final class CanMatchPreFilterSearchPhase { this.shardItIndexMap = shardItIndexMap; } + public static void execute( + Logger logger, + SearchTransportService searchTransportService, + BiFunction nodeIdToConnection, + Map aliasFilter, + Map concreteIndexBoosts, + Executor executor, + SearchRequest request, + GroupShardsIterator shardsIts, + TransportSearchAction.SearchTimeProvider timeProvider, + SearchTask task, + boolean requireAtLeastOneMatch, + CoordinatorRewriteContextProvider coordinatorRewriteContextProvider, + ActionListener> listener + ) { + if (shardsIts.size() == 0) { + listener.onResponse(shardsIts); + return; + } + new CanMatchPreFilterSearchPhase( + logger, + searchTransportService, + nodeIdToConnection, + aliasFilter, + concreteIndexBoosts, + executor, + request, + shardsIts, + timeProvider, + task, + requireAtLeastOneMatch, + coordinatorRewriteContextProvider, + listener + ).start(); + } + private static boolean assertSearchCoordinationThread() { return ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SEARCH_COORDINATION); } @@ -134,34 +172,35 @@ private void runCoordinatorRewritePhase() { assert assertSearchCoordinationThread(); final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { - final CanMatchNodeRequest canMatchNodeRequest = new CanMatchNodeRequest( - request, - searchShardIterator.getOriginalIndices().indicesOptions(), - Collections.emptyList(), - getNumShards(), - timeProvider.absoluteStartMillis(), - searchShardIterator.getClusterAlias() - ); - final ShardSearchRequest request = canMatchNodeRequest.createShardSearchRequest(buildShardLevelRequest(searchShardIterator)); if (searchShardIterator.prefiltered()) { - consumeResult(searchShardIterator.skip() == false, request); + if (searchShardIterator.skip() == false) { + res.set(shardItIndexMap.get(searchShardIterator), TRUE_SENTINEL); + } continue; } boolean canMatch = true; CoordinatorRewriteContext coordinatorRewriteContext = coordinatorRewriteContextProvider.getCoordinatorRewriteContext( - request.shardId().getIndex() + searchShardIterator.shardId().getIndex() ); if (coordinatorRewriteContext != null) { try { - canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); + canMatch = SearchService.queryStillMatchesAfterRewrite( + new CanMatchNodeRequest( + request, + searchShardIterator.getOriginalIndices().indicesOptions(), + Collections.emptyList(), + shardsIts.size(), + timeProvider.absoluteStartMillis(), + searchShardIterator.getClusterAlias() + ).createShardSearchRequest(buildShardLevelRequest(searchShardIterator)), + coordinatorRewriteContext + ); } catch (Exception e) { // treat as if shard is still a potential match } } if (canMatch) { matchedShardLevelRequests.add(searchShardIterator); - } else { - consumeResult(false, request); } } if (matchedShardLevelRequests.isEmpty()) { @@ -169,118 +208,99 @@ private void runCoordinatorRewritePhase() { } else { GroupShardsIterator matchingShards = new GroupShardsIterator<>(matchedShardLevelRequests); // verify missing shards only for the shards that we hit for the query - checkNoMissingShards(matchingShards); + SearchPhase.doCheckNoMissingShards("can_match", request, matchingShards, SearchPhase::makeMissingShardsError); new Round(matchingShards).run(); } } - private void consumeResult(boolean canMatch, ShardSearchRequest request) { - CanMatchShardResponse result = new CanMatchShardResponse(canMatch, null); - result.setShardIndex(request.shardRequestIndex()); - results.consumeResult(result, () -> {}); - } - - private void checkNoMissingShards(GroupShardsIterator shards) { - assert assertSearchCoordinationThread(); - SearchPhase.doCheckNoMissingShards("can_match", request, shards, SearchPhase::makeMissingShardsError); - } - - private Map> groupByNode(GroupShardsIterator shards) { - Map> requests = new HashMap<>(); - for (int i = 0; i < shards.size(); i++) { - final SearchShardIterator shardRoutings = shards.get(i); - assert shardRoutings.skip() == false; - assert shardItIndexMap.containsKey(shardRoutings); - SearchShardTarget target = shardRoutings.nextOrNull(); - if (target != null) { - requests.computeIfAbsent(new SendingTarget(target.getClusterAlias(), target.getNodeId()), t -> new ArrayList<>()) - .add(shardRoutings); - } else { - requests.computeIfAbsent(new SendingTarget(null, null), t -> new ArrayList<>()).add(shardRoutings); - } - } - return requests; - } - /** * Sending can-match requests is round-based and grouped per target node. * If there are failures during a round, there will be a follow-up round * to retry on other available shard copies. */ - class Round extends AbstractRunnable { + private final class Round extends AbstractRunnable { private final GroupShardsIterator shards; private final CountDown countDown; - private final AtomicReferenceArray failedResponses; Round(GroupShardsIterator shards) { this.shards = shards; this.countDown = new CountDown(shards.size()); - this.failedResponses = new AtomicReferenceArray<>(shardsIts.size()); } @Override protected void doRun() { assert assertSearchCoordinationThread(); - final Map> requests = groupByNode(shards); + Map> requests = new HashMap<>(); + for (int i = 0; i < shards.size(); i++) { + final SearchShardIterator shardRoutings = shards.get(i); + assert shardRoutings.skip() == false; + assert shardItIndexMap.containsKey(shardRoutings); + SearchShardTarget target = shardRoutings.nextOrNull(); + if (target != null) { + requests.computeIfAbsent(new SendingTarget(target.getClusterAlias(), target.getNodeId()), t -> new ArrayList<>()) + .add(shardRoutings); + } else { + onOperationResult(shardItIndexMap.get(shardRoutings), TRUE_SENTINEL); + } + } for (Map.Entry> entry : requests.entrySet()) { - CanMatchNodeRequest canMatchNodeRequest = createCanMatchRequest(entry); + CanMatchNodeRequest canMatchNodeRequest = createCanMatchRequest(entry.getValue()); List shardLevelRequests = canMatchNodeRequest.getShardLevelRequests(); - - if (entry.getKey().nodeId == null) { - // no target node: just mark the requests as failed - for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { - onOperationFailed(shard.getShardRequestIndex(), null); - } - continue; - } - + var sendingTarget = entry.getKey(); try { - searchTransportService.sendCanMatch(getConnection(entry.getKey()), canMatchNodeRequest, task, new ActionListener<>() { - @Override - public void onResponse(CanMatchNodeResponse canMatchNodeResponse) { - assert canMatchNodeResponse.getResponses().size() == canMatchNodeRequest.getShardLevelRequests().size(); - for (int i = 0; i < canMatchNodeResponse.getResponses().size(); i++) { - CanMatchNodeResponse.ResponseOrFailure response = canMatchNodeResponse.getResponses().get(i); - if (response.getResponse() != null) { + searchTransportService.sendCanMatch( + nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId), + canMatchNodeRequest, + task, + new ActionListener<>() { + @Override + public void onResponse(CanMatchNodeResponse canMatchNodeResponse) { + assert canMatchNodeResponse.getResponses().size() == shardLevelRequests.size(); + for (int i = 0; i < canMatchNodeResponse.getResponses().size(); i++) { + CanMatchNodeResponse.ResponseOrFailure response = canMatchNodeResponse.getResponses().get(i); CanMatchShardResponse shardResponse = response.getResponse(); - shardResponse.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); - onOperation(shardResponse.getShardIndex(), shardResponse); - } else { - Exception failure = response.getException(); - assert failure != null; - onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); + final Object shardResult; + if (shardResponse != null) { + shardResult = shardResponse.canMatch() + ? Objects.requireNonNullElse(shardResponse.estimatedMinAndMax(), TRUE_SENTINEL) + : null; + } else { + Exception failure = response.getException(); + assert failure != null; + shardResult = failure; + } + onOperationResult(shardLevelRequests.get(i).getShardRequestIndex(), shardResult); } } - } - @Override - public void onFailure(Exception e) { - for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { - onOperationFailed(shard.getShardRequestIndex(), e); + @Override + public void onFailure(Exception e) { + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { + onOperationResult(shard.getShardRequestIndex(), e); + } } } - }); + ); } catch (Exception e) { for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { - onOperationFailed(shard.getShardRequestIndex(), e); + onOperationResult(shard.getShardRequestIndex(), e); } } } } - private void onOperation(int idx, CanMatchShardResponse response) { - failedResponses.set(idx, null); - results.consumeResult(response, () -> { - if (countDown.countDown()) { - finishRound(); - } - }); + @Override + public boolean isForceExecution() { + return true; + } + + private void onOperationResult(int idx, Object result) { + res.set(idx, result); + maybeFinish(); } - private void onOperationFailed(int idx, Exception e) { - failedResponses.set(idx, e); - results.consumeShardFailure(idx); + private void maybeFinish() { if (countDown.countDown()) { finishRound(); } @@ -289,8 +309,8 @@ private void onOperationFailed(int idx, Exception e) { private void finishRound() { List remainingShards = new ArrayList<>(); for (SearchShardIterator ssi : shards) { - int shardIndex = shardItIndexMap.get(ssi); - Exception failedResponse = failedResponses.get(shardIndex); + int idx = shardItIndexMap.get(ssi); + Exception failedResponse = res.get(idx) instanceof Exception e ? e : null; if (failedResponse != null) { remainingShards.add(ssi); } @@ -299,49 +319,38 @@ private void finishRound() { finishPhase(); } else { // trigger another round, forcing execution - executor.execute(new Round(new GroupShardsIterator<>(remainingShards)) { - @Override - public boolean isForceExecution() { - return true; - } - }); + executor.execute(new Round(new GroupShardsIterator<>(remainingShards))); } } @Override public void onFailure(Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(() -> format("Failed to execute [%s] while running [can_match] phase", request), e); - } onPhaseFailure("round", e); } } private record SendingTarget(@Nullable String clusterAlias, @Nullable String nodeId) {} - private CanMatchNodeRequest createCanMatchRequest(Map.Entry> entry) { - final SearchShardIterator first = entry.getValue().get(0); - final List shardLevelRequests = entry.getValue() - .stream() - .map(this::buildShardLevelRequest) - .collect(Collectors.toCollection(ArrayList::new)); - assert entry.getValue().stream().allMatch(Objects::nonNull); - assert entry.getValue() - .stream() - .allMatch(ssi -> Objects.equals(ssi.getOriginalIndices().indicesOptions(), first.getOriginalIndices().indicesOptions())); - assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); + private CanMatchNodeRequest createCanMatchRequest(List iters) { + final SearchShardIterator first = iters.get(0); + assert iters.stream() + .allMatch( + ssi -> ssi != null + && Objects.equals(ssi.getOriginalIndices().indicesOptions(), first.getOriginalIndices().indicesOptions()) + && Objects.equals(ssi.getClusterAlias(), first.getClusterAlias()) + ); return new CanMatchNodeRequest( request, first.getOriginalIndices().indicesOptions(), - shardLevelRequests, - getNumShards(), + iters.stream().map(this::buildShardLevelRequest).toList(), + shardsIts.size(), timeProvider.absoluteStartMillis(), first.getClusterAlias() ); } private void finishPhase() { - listener.onResponse(getIterator(results, shardsIts)); + listener.onResponse(getIterator()); } private static final float DEFAULT_INDEX_BOOST = 1.0f; @@ -363,18 +372,11 @@ public CanMatchNodeRequest.Shard buildShardLevelRequest(SearchShardIterator shar ); } - public void start() { - if (getNumShards() == 0) { - finishPhase(); - return; - } + private void start() { // Note that the search is failed when this task is rejected by the executor executor.execute(new AbstractRunnable() { @Override public void onFailure(Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(() -> format("Failed to execute [%s] while running [can_match] phase", request), e); - } onPhaseFailure("start", e); } @@ -386,131 +388,72 @@ protected void doRun() { }); } - public void onPhaseFailure(String msg, Exception cause) { - listener.onFailure(new SearchPhaseExecutionException("can_match", msg, cause, ShardSearchFailure.EMPTY_ARRAY)); - } - - public Transport.Connection getConnection(SendingTarget sendingTarget) { - return nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId); - } - - private int getNumShards() { - return shardsIts.size(); - } - - private static final class CanMatchSearchPhaseResults extends SearchPhaseResults { - private final FixedBitSet possibleMatches; - private final MinAndMax[] minAndMaxes; - private int numPossibleMatches; - - CanMatchSearchPhaseResults(int size) { - super(size); - possibleMatches = new FixedBitSet(size); - minAndMaxes = new MinAndMax[size]; - } - - @Override - void consumeResult(CanMatchShardResponse result, Runnable next) { - try { - final boolean canMatch = result.canMatch(); - final MinAndMax minAndMax = result.estimatedMinAndMax(); - if (canMatch || minAndMax != null) { - consumeResult(result.getShardIndex(), canMatch, minAndMax); - } - } finally { - next.run(); - } - } - - @Override - boolean hasResult(int shardIndex) { - return false; // unneeded + private void onPhaseFailure(String msg, Exception cause) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while running [can_match] phase", request), cause); } - - @Override - void consumeShardFailure(int shardIndex) { - // we have to carry over shard failures in order to account for them in the response. - consumeResult(shardIndex, true, null); - } - - private synchronized void consumeResult(int shardIndex, boolean canMatch, MinAndMax minAndMax) { - if (canMatch) { - possibleMatches.set(shardIndex); - numPossibleMatches++; - } - minAndMaxes[shardIndex] = minAndMax; - } - - synchronized int getNumPossibleMatches() { - return numPossibleMatches; - } - - synchronized FixedBitSet getPossibleMatches() { - return possibleMatches; - } - - @Override - Stream getSuccessfulResults() { - return Stream.empty(); - } - - @Override - public void close() {} + listener.onFailure(new SearchPhaseExecutionException("can_match", msg, cause, ShardSearchFailure.EMPTY_ARRAY)); } - private GroupShardsIterator getIterator( - CanMatchSearchPhaseResults results, - GroupShardsIterator shardsIts - ) { - FixedBitSet possibleMatches = results.getPossibleMatches(); + private GroupShardsIterator getIterator() { // TODO: pick the local shard when possible - if (requireAtLeastOneMatch && results.getNumPossibleMatches() == 0) { - // this is a special case where we have no hit but we need to get at least one search response in order - // to produce a valid search result with all the aggs etc. - // Since it's possible that some of the shards that we're skipping are - // unavailable, we would try to query the node that at least has some - // shards available in order to produce a valid search result. - int shardIndexToQuery = 0; - for (int i = 0; i < shardsIts.size(); i++) { - SearchShardIterator it = shardsIts.get(i); - if (it.size() > 0) { - shardIndexToQuery = i; - it.skip(false); // un-skip which is needed when all the remote shards were skipped by the remote can_match + if (requireAtLeastOneMatch) { + boolean isEmpty = true; + for (int i = 0, n = res.length(); i < n; i++) { + if (res.get(i) != null) { + isEmpty = false; break; } } - possibleMatches.set(shardIndexToQuery); + if (isEmpty) { + // this is a special case where we have no hit, but we need to get at least one search response in order + // to produce a valid search result with all the aggs etc. + // Since it's possible that some of the shards that we're skipping are + // unavailable, we would try to query the node that at least has some + // shards available in order to produce a valid search result. + int shardIndexToQuery = 0; + for (int i = 0; i < shardsIts.size(); i++) { + SearchShardIterator it = shardsIts.get(i); + if (it.size() > 0) { + shardIndexToQuery = i; + it.skip(false); // un-skip which is needed when all the remote shards were skipped by the remote can_match + break; + } + } + res.set(shardIndexToQuery, TRUE_SENTINEL); + } } int i = 0; for (SearchShardIterator iter : shardsIts) { iter.reset(); - boolean match = possibleMatches.get(i++); - if (match) { + if (res.get(i++) != null) { assert iter.skip() == false; } else { iter.skip(true); } } - if (shouldSortShards(results.minAndMaxes) == false) { - return shardsIts; - } - FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); - return new GroupShardsIterator<>(sortShards(shardsIts, results.minAndMaxes, fieldSort.order())); + return shouldSortShards(res) + ? new GroupShardsIterator<>(sortShards(shardsIts, res, FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()).order())) + : shardsIts; + } + + private static MinAndMax getMinAndMax(AtomicReferenceArray res, int shardIndex) { + return res.get(shardIndex) instanceof MinAndMax m ? m : null; } private static List sortShards( GroupShardsIterator shardsIts, - MinAndMax[] minAndMaxes, + AtomicReferenceArray results, SortOrder order ) { int bound = shardsIts.size(); - List toSort = new ArrayList<>(bound); + Integer[] toSort = new Integer[bound]; for (int i = 0; i < bound; i++) { - toSort.add(i); + toSort[i] = i; } Comparator> keyComparator = forciblyCast(MinAndMax.getComparator(order)); - toSort.sort((idx1, idx2) -> { - int res = keyComparator.compare(minAndMaxes[idx1], minAndMaxes[idx2]); + Arrays.sort(toSort, (idx1, idx2) -> { + int res = keyComparator.compare(getMinAndMax(results, idx1), getMinAndMax(results, idx2)); if (res != 0) { return res; } @@ -523,9 +466,10 @@ private static List sortShards( return list; } - private static boolean shouldSortShards(MinAndMax[] minAndMaxes) { + private static boolean shouldSortShards(AtomicReferenceArray results) { Class clazz = null; - for (MinAndMax minAndMax : minAndMaxes) { + for (int i = 0, n = results.length(); i < n; i++) { + var minAndMax = getMinAndMax(results, i); if (clazz == null) { clazz = minAndMax == null ? null : minAndMax.getMin().getClass(); } else if (minAndMax != null && clazz != minAndMax.getMin().getClass()) { diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java index 7ba4a7ce59869..4c17cf5cee3b6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java @@ -166,7 +166,7 @@ public void runNewSearchPhase( // that is signaled to the local can match through the SearchShardIterator#prefiltered flag. Local shards do need to go // through the local can match phase. if (SearchService.canRewriteToMatchNone(searchRequest.source())) { - new CanMatchPreFilterSearchPhase( + CanMatchPreFilterSearchPhase.execute( logger, searchTransportService, connectionLookup, @@ -193,7 +193,7 @@ public void runNewSearchPhase( clusters ) ) - ).start(); + ); } else { runOpenPointInTimePhase( task, diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 4bca7a562fc38..cfdb8b4ed4191 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -1455,7 +1455,7 @@ public void runNewSearchPhase( if (preFilter) { // only for aggs we need to contact shards even if there are no matches boolean requireAtLeastOneMatch = searchRequest.source() != null && searchRequest.source().aggregations() != null; - new CanMatchPreFilterSearchPhase( + CanMatchPreFilterSearchPhase.execute( logger, searchTransportService, connectionLookup, @@ -1468,8 +1468,8 @@ public void runNewSearchPhase( task, requireAtLeastOneMatch, searchService.getCoordinatorRewriteContextProvider(timeProvider::absoluteStartMillis), - listener.delegateFailureAndWrap((l, iters) -> { - runNewSearchPhase( + listener.delegateFailureAndWrap( + (l, iters) -> runNewSearchPhase( task, searchRequest, executor, @@ -1482,9 +1482,9 @@ public void runNewSearchPhase( false, threadPool, clusters - ); - }) - ).start(); + ) + ) + ); return; } // for synchronous CCS minimize_roundtrips=false, use the CCSSingleCoordinatorSearchProgressListener diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java index d8b57972d604f..fbf582fe26993 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java @@ -146,7 +146,7 @@ public void searchShards(Task task, SearchShardsRequest searchShardsRequest, Act if (SearchService.canRewriteToMatchNone(searchRequest.source()) == false) { delegate.onResponse(new SearchShardsResponse(toGroups(shardIts), clusterState.nodes().getAllNodes(), aliasFilters)); } else { - new CanMatchPreFilterSearchPhase(logger, searchTransportService, (clusterAlias, node) -> { + CanMatchPreFilterSearchPhase.execute(logger, searchTransportService, (clusterAlias, node) -> { assert Objects.equals(clusterAlias, searchShardsRequest.clusterAlias()); return transportService.getConnection(clusterState.nodes().get(node)); }, @@ -160,7 +160,7 @@ public void searchShards(Task task, SearchShardsRequest searchShardsRequest, Act false, searchService.getCoordinatorRewriteContextProvider(timeProvider::absoluteStartMillis), delegate.map(its -> new SearchShardsResponse(toGroups(its), clusterState.nodes().getAllNodes(), aliasFilters)) - ).start(); + ); } }) ); diff --git a/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java index 985e7a52aae81..cb6983c90292a 100644 --- a/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java +++ b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.search.sort.MinAndMax; import java.io.IOException; @@ -18,12 +19,11 @@ /** * Shard-level response for can-match requests */ -public final class CanMatchShardResponse extends SearchPhaseResult { +public final class CanMatchShardResponse implements Writeable { private final boolean canMatch; private final MinAndMax estimatedMinAndMax; public CanMatchShardResponse(StreamInput in) throws IOException { - super(in); this.canMatch = in.readBoolean(); estimatedMinAndMax = in.readOptionalWriteable(MinAndMax::new); } diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index 69872b5e4b546..235935371a5b1 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -149,7 +149,7 @@ public void sendCanMatch( final SearchRequest searchRequest = new SearchRequest(); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( + CanMatchPreFilterSearchPhase.execute( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -167,8 +167,6 @@ public void sendCanMatch( latch.countDown(); }) ); - - canMatchPhase.start(); latch.await(); assertThat(numRequests.get(), replicaNode == null ? equalTo(1) : lessThanOrEqualTo(2)); @@ -247,7 +245,7 @@ public void sendCanMatch( final SearchRequest searchRequest = new SearchRequest(); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( + CanMatchPreFilterSearchPhase.execute( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -265,8 +263,6 @@ public void sendCanMatch( latch.countDown(); }) ); - - canMatchPhase.start(); latch.await(); assertEquals(0, result.get().get(0).shardId().id()); @@ -309,8 +305,8 @@ public void sendCanMatch( for (CanMatchNodeRequest.Shard shard : request.getShardLevelRequests()) { Long min = rarely() ? null : randomLong(); Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); - MinAndMax minMax = min == null ? null : new MinAndMax<>(min, max); boolean canMatch = frequently(); + MinAndMax minMax = canMatch == false || min == null ? null : new MinAndMax<>(min, max); synchronized (shardIds) { shardIds.add(shard.shardId()); minAndMaxes.add(minMax); @@ -340,7 +336,7 @@ public void sendCanMatch( searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( + CanMatchPreFilterSearchPhase.execute( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -358,8 +354,6 @@ public void sendCanMatch( latch.countDown(); }) ); - - canMatchPhase.start(); latch.await(); ShardId[] expected = IntStream.range(0, shardIds.size()) .boxed() @@ -442,7 +436,7 @@ public void sendCanMatch( searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( + CanMatchPreFilterSearchPhase.execute( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -460,8 +454,6 @@ public void sendCanMatch( latch.countDown(); }) ); - - canMatchPhase.start(); latch.await(); int shardId = 0; for (SearchShardIterator i : result.get()) { @@ -1192,7 +1184,7 @@ public void testCanMatchFilteringOnCoordinatorWithMissingShards() throws Excepti // test that a search does fail if the query does NOT filter ALL the // unassigned shards CountDownLatch latch = new CountDownLatch(1); - Tuple> canMatchPhaseAndRequests = getCanMatchPhaseAndRequests( + Tuple> canMatchPhaseAndRequests = getCanMatchPhaseAndRequests( List.of(dataStream), List.of(hotRegularIndex, warmRegularIndex), coordinatorRewriteContextProvider, @@ -1216,7 +1208,7 @@ public void onFailure(Exception e) { } ); - canMatchPhaseAndRequests.v1().start(); + canMatchPhaseAndRequests.v1().run(); latch.await(10, TimeUnit.SECONDS); } } @@ -1271,7 +1263,7 @@ private void assignShardsAndExecuteCanMatchPhase( ) throws Exception { AtomicReference> result = new AtomicReference<>(); CountDownLatch latch = new CountDownLatch(1); - Tuple> canMatchAndShardRequests = getCanMatchPhaseAndRequests( + Tuple> canMatchAndShardRequests = getCanMatchPhaseAndRequests( dataStreams, regularIndices, contextProvider, @@ -1286,7 +1278,7 @@ private void assignShardsAndExecuteCanMatchPhase( }) ); - canMatchAndShardRequests.v1().start(); + canMatchAndShardRequests.v1().run(); latch.await(); List updatedSearchShardIterators = new ArrayList<>(); @@ -1297,7 +1289,7 @@ private void assignShardsAndExecuteCanMatchPhase( canMatchResultsConsumer.accept(updatedSearchShardIterators, canMatchAndShardRequests.v2()); } - private Tuple> getCanMatchPhaseAndRequests( + private Tuple> getCanMatchPhaseAndRequests( List dataStreams, List regularIndices, CoordinatorRewriteContextProvider contextProvider, @@ -1416,9 +1408,8 @@ public void sendCanMatch( System::nanoTime ); - AtomicReference> result = new AtomicReference<>(); return new Tuple<>( - new CanMatchPreFilterSearchPhase( + () -> CanMatchPreFilterSearchPhase.execute( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), From ac4c6e02818f899e26dbf49c00c675e1230deedf Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 16 Nov 2024 09:50:25 +0100 Subject: [PATCH 003/132] simple can-match step 4 --- .../search/CanMatchPreFilterSearchPhase.java | 66 +++++++++---------- 1 file changed, 31 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index b28bf6457b970..dd2258eebf575 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -248,44 +248,44 @@ protected void doRun() { CanMatchNodeRequest canMatchNodeRequest = createCanMatchRequest(entry.getValue()); List shardLevelRequests = canMatchNodeRequest.getShardLevelRequests(); var sendingTarget = entry.getKey(); + var listener = new ActionListener() { + @Override + public void onResponse(CanMatchNodeResponse canMatchNodeResponse) { + var responses = canMatchNodeResponse.getResponses(); + assert responses.size() == shardLevelRequests.size(); + for (int i = 0; i < responses.size(); i++) { + CanMatchNodeResponse.ResponseOrFailure response = responses.get(i); + CanMatchShardResponse shardResponse = response.getResponse(); + final Object shardResult; + if (shardResponse != null) { + shardResult = shardResponse.canMatch() + ? Objects.requireNonNullElse(shardResponse.estimatedMinAndMax(), TRUE_SENTINEL) + : null; + } else { + Exception failure = response.getException(); + assert failure != null; + shardResult = failure; + } + onOperationResult(shardLevelRequests.get(i).getShardRequestIndex(), shardResult); + } + } + + @Override + public void onFailure(Exception e) { + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { + onOperationResult(shard.getShardRequestIndex(), e); + } + } + }; try { searchTransportService.sendCanMatch( nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId), canMatchNodeRequest, task, - new ActionListener<>() { - @Override - public void onResponse(CanMatchNodeResponse canMatchNodeResponse) { - assert canMatchNodeResponse.getResponses().size() == shardLevelRequests.size(); - for (int i = 0; i < canMatchNodeResponse.getResponses().size(); i++) { - CanMatchNodeResponse.ResponseOrFailure response = canMatchNodeResponse.getResponses().get(i); - CanMatchShardResponse shardResponse = response.getResponse(); - final Object shardResult; - if (shardResponse != null) { - shardResult = shardResponse.canMatch() - ? Objects.requireNonNullElse(shardResponse.estimatedMinAndMax(), TRUE_SENTINEL) - : null; - } else { - Exception failure = response.getException(); - assert failure != null; - shardResult = failure; - } - onOperationResult(shardLevelRequests.get(i).getShardRequestIndex(), shardResult); - } - } - - @Override - public void onFailure(Exception e) { - for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { - onOperationResult(shard.getShardRequestIndex(), e); - } - } - } + listener ); } catch (Exception e) { - for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { - onOperationResult(shard.getShardRequestIndex(), e); - } + listener.onFailure(e); } } } @@ -297,10 +297,6 @@ public boolean isForceExecution() { private void onOperationResult(int idx, Object result) { res.set(idx, result); - maybeFinish(); - } - - private void maybeFinish() { if (countDown.countDown()) { finishRound(); } From b6c94e1688750a5072468eeabd353b090cf32a95 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 20 Nov 2024 15:56:19 +0100 Subject: [PATCH 004/132] Make shardIndexMap in AbstractSearchAsyncAction a local variable We only need this rather large map in `run`, lets create it on the fly there to save the rather large redundant field and save on state and complexity in general. --- .../search/AbstractSearchAsyncAction.java | 22 ++++++++----------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 09fb70fb06ba4..2889f0a8e7c80 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -10,7 +10,6 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; -import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; @@ -26,6 +25,7 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -43,8 +43,7 @@ import org.elasticsearch.transport.Transport; import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -98,7 +97,6 @@ abstract class AbstractSearchAsyncAction exten protected final GroupShardsIterator toSkipShardsIts; protected final GroupShardsIterator shardsIts; private final SearchShardIterator[] shardIterators; - private final Map shardIndexMap; private final int expectedTotalOps; private final AtomicInteger totalOps = new AtomicInteger(); private final int maxConcurrentRequestsPerNode; @@ -142,17 +140,11 @@ abstract class AbstractSearchAsyncAction exten this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); this.shardsIts = new GroupShardsIterator<>(iterators); - // we compute the shard index based on the natural order of the shards + this.shardIterators = iterators.toArray(new SearchShardIterator[0]); + // we later compute the shard index based on the natural order of the shards // that participate in the search request. This means that this number is // consistent between two requests that target the same shards. - Map shardMap = new HashMap<>(); - List searchIterators = new ArrayList<>(iterators); - CollectionUtil.timSort(searchIterators); - for (int i = 0; i < searchIterators.size(); i++) { - shardMap.put(searchIterators.get(i), i); - } - this.shardIndexMap = Collections.unmodifiableMap(shardMap); - this.shardIterators = searchIterators.toArray(SearchShardIterator[]::new); + Arrays.sort(shardIterators); // we need to add 1 for non active partition, since we count it in the total. This means for each shard in the iterator we sum up // it's number of active shards but use 1 as the default if no replica of a shard is active at this point. @@ -236,6 +228,10 @@ public final void run() { assert iterator.skip(); skipShard(iterator); } + final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); + for (int i = 0; i < shardIterators.length; i++) { + shardIndexMap.put(shardIterators[i], i); + } if (shardsIts.size() > 0) { doCheckNoMissingShards(getName(), request, shardsIts); for (int i = 0; i < shardsIts.size(); i++) { From 67ff8d393e953c370154aa68d1705ab307d7ef07 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 21 Nov 2024 13:23:18 +0100 Subject: [PATCH 005/132] Remove needless sending of OriginalIndices in SearchFreeContextRequest We don't need to use this request, the handler for freeing of scroll requests literally goes to the same transport handler and doesn't come with the list of indices. The original security need for keeping the list of indices around is long gone. --- .../action/IndicesRequestIT.java | 13 +--- .../search/AbstractSearchAsyncAction.java | 6 +- .../action/search/DfsQueryPhase.java | 6 +- .../action/search/SearchPhase.java | 6 +- .../action/search/SearchTransportService.java | 70 ++----------------- .../AbstractSearchAsyncActionTests.java | 6 +- .../action/search/MockSearchPhaseContext.java | 2 +- .../action/search/SearchAsyncActionTests.java | 15 +++- .../test/ESSingleNodeTestCase.java | 4 +- 9 files changed, 30 insertions(+), 98 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java index 8bedf436e3698..f5860cedcd989 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java @@ -556,11 +556,7 @@ public void testUpdateSettings() { } public void testSearchQueryThenFetch() throws Exception { - interceptTransportActions( - SearchTransportService.QUERY_ACTION_NAME, - SearchTransportService.FETCH_ID_ACTION_NAME, - SearchTransportService.FREE_CONTEXT_ACTION_NAME - ); + interceptTransportActions(SearchTransportService.QUERY_ACTION_NAME, SearchTransportService.FETCH_ID_ACTION_NAME); String[] randomIndicesOrAliases = randomIndicesOrAliases(); for (int i = 0; i < randomIndicesOrAliases.length; i++) { @@ -580,16 +576,13 @@ public void testSearchQueryThenFetch() throws Exception { SearchTransportService.QUERY_ACTION_NAME, SearchTransportService.FETCH_ID_ACTION_NAME ); - // free context messages are not necessarily sent, but if they are, check their indices - assertIndicesSubsetOptionalRequests(Arrays.asList(searchRequest.indices()), SearchTransportService.FREE_CONTEXT_ACTION_NAME); } public void testSearchDfsQueryThenFetch() throws Exception { interceptTransportActions( SearchTransportService.DFS_ACTION_NAME, SearchTransportService.QUERY_ID_ACTION_NAME, - SearchTransportService.FETCH_ID_ACTION_NAME, - SearchTransportService.FREE_CONTEXT_ACTION_NAME + SearchTransportService.FETCH_ID_ACTION_NAME ); String[] randomIndicesOrAliases = randomIndicesOrAliases(); @@ -611,8 +604,6 @@ public void testSearchDfsQueryThenFetch() throws Exception { SearchTransportService.QUERY_ID_ACTION_NAME, SearchTransportService.FETCH_ID_ACTION_NAME ); - // free context messages are not necessarily sent, but if they are, check their indices - assertIndicesSubsetOptionalRequests(Arrays.asList(searchRequest.indices()), SearchTransportService.FREE_CONTEXT_ACTION_NAME); } private static void assertSameIndices(IndicesRequest originalRequest, String... actions) { diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 09fb70fb06ba4..c1d45d7eba97d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -711,7 +711,7 @@ private void raisePhaseFailure(SearchPhaseExecutionException exception) { try { SearchShardTarget searchShardTarget = entry.getSearchShardTarget(); Transport.Connection connection = getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId()); - sendReleaseSearchContext(entry.getContextId(), connection, getOriginalIndices(entry.getShardIndex())); + sendReleaseSearchContext(entry.getContextId(), connection); } catch (Exception inner) { inner.addSuppressed(exception); logger.trace("failed to release context", inner); @@ -727,10 +727,10 @@ private void raisePhaseFailure(SearchPhaseExecutionException exception) { * @see org.elasticsearch.search.fetch.FetchSearchResult#getContextId() * */ - void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection, OriginalIndices originalIndices) { + void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { assert isPartOfPointInTime(contextId) == false : "Must not release point in time context [" + contextId + "]"; if (connection != null) { - searchTransportService.sendFreeContext(connection, contextId, originalIndices); + searchTransportService.sendFreeContext(connection, contextId, ActionListener.noop()); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java index 285dd0a22fd7e..cc8c4becea9a9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java @@ -119,11 +119,7 @@ public void onFailure(Exception exception) { // the query might not have been executed at all (for example because thread pool rejected // execution) and the search context that was created in dfs phase might not be released. // release it again to be in the safe side - context.sendReleaseSearchContext( - querySearchRequest.contextId(), - connection, - context.getOriginalIndices(shardIndex) - ); + context.sendReleaseSearchContext(querySearchRequest.contextId(), connection); } } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index 986f7210c0d1b..7d849a72abf9d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -97,11 +97,7 @@ protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPha context.getLogger().trace("trying to release search context [{}]", phaseResult.getContextId()); SearchShardTarget shardTarget = phaseResult.getSearchShardTarget(); Transport.Connection connection = context.getConnection(shardTarget.getClusterAlias(), shardTarget.getNodeId()); - context.sendReleaseSearchContext( - phaseResult.getContextId(), - connection, - context.getOriginalIndices(phaseResult.getShardIndex()) - ); + context.sendReleaseSearchContext(phaseResult.getContextId(), connection); } catch (Exception e) { context.getLogger().trace("failed to release context", e); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 8444a92b24432..cfc2e1bcdaf2b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -13,12 +13,10 @@ import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListenerResponseHandler; -import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.admin.cluster.node.tasks.get.TransportGetTaskAction; import org.elasticsearch.action.support.ChannelActionListener; -import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.client.internal.node.NodeClient; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -124,24 +122,6 @@ public SearchTransportService( this.responseWrapper = responseWrapper; } - private static final ActionListenerResponseHandler SEND_FREE_CONTEXT_LISTENER = - new ActionListenerResponseHandler<>( - ActionListener.noop(), - SearchFreeContextResponse::readFrom, - TransportResponseHandler.TRANSPORT_WORKER - ); - - public void sendFreeContext(Transport.Connection connection, final ShardSearchContextId contextId, OriginalIndices originalIndices) { - transportService.sendRequest( - connection, - FREE_CONTEXT_ACTION_NAME, - new SearchFreeContextRequest(originalIndices, contextId), - TransportRequestOptions.EMPTY, - // no need to respond if it was freed or not - SEND_FREE_CONTEXT_LISTENER - ); - } - public void sendFreeContext( Transport.Connection connection, ShardSearchContextId contextId, @@ -370,43 +350,6 @@ private static class ClearScrollContextsRequest extends TransportRequest { } } - static class SearchFreeContextRequest extends ScrollFreeContextRequest implements IndicesRequest { - private final OriginalIndices originalIndices; - - SearchFreeContextRequest(OriginalIndices originalIndices, ShardSearchContextId id) { - super(id); - this.originalIndices = originalIndices; - } - - SearchFreeContextRequest(StreamInput in) throws IOException { - super(in); - originalIndices = OriginalIndices.readOriginalIndices(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - OriginalIndices.writeOriginalIndices(originalIndices, out); - } - - @Override - public String[] indices() { - if (originalIndices == null) { - return null; - } - return originalIndices.indices(); - } - - @Override - public IndicesOptions indicesOptions() { - if (originalIndices == null) { - return null; - } - return originalIndices.indicesOptions(); - } - - } - public static class SearchFreeContextResponse extends TransportResponse { private static final SearchFreeContextResponse FREED = new SearchFreeContextResponse(true); @@ -456,12 +399,13 @@ public static void registerRequestHandler(TransportService transportService, Sea SearchFreeContextResponse::readFrom ); - transportService.registerRequestHandler( - FREE_CONTEXT_ACTION_NAME, - freeContextExecutor, - SearchFreeContextRequest::new, - freeContextHandler - ); + // TODO: remove this handler once the lowest compatible version stops using it + transportService.registerRequestHandler(FREE_CONTEXT_ACTION_NAME, freeContextExecutor, in -> { + var res = new ScrollFreeContextRequest(in); + // this handler exists for BwC purposes only, we don't need the original indices to free the context + OriginalIndices.readOriginalIndices(in); + return res; + }, freeContextHandler); TransportActionProxy.registerProxyAction(transportService, FREE_CONTEXT_ACTION_NAME, false, SearchFreeContextResponse::readFrom); transportService.registerRequestHandler( diff --git a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java index 725a4583d104a..71bf2a47cfa47 100644 --- a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java @@ -112,11 +112,7 @@ long buildTookInMillis() { } @Override - public void sendReleaseSearchContext( - ShardSearchContextId contextId, - Transport.Connection connection, - OriginalIndices originalIndices - ) { + public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { releasedContexts.add(contextId); } diff --git a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java index 484b3c6b386fd..1f165c5b9337e 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java +++ b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java @@ -160,7 +160,7 @@ public void onFailure(Exception e) { } @Override - public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection, OriginalIndices originalIndices) { + public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { releasedSearchContexts.add(contextId); } diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index b4ddd48172d01..2361beb7ad036 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -296,7 +296,11 @@ public void testFanOutAndCollect() throws InterruptedException { AtomicInteger numFreedContext = new AtomicInteger(); SearchTransportService transportService = new SearchTransportService(null, null, null) { @Override - public void sendFreeContext(Transport.Connection connection, ShardSearchContextId contextId, OriginalIndices originalIndices) { + public void sendFreeContext( + Transport.Connection connection, + ShardSearchContextId contextId, + ActionListener listener + ) { numFreedContext.incrementAndGet(); assertTrue(nodeToContextMap.containsKey(connection.getNode())); assertTrue(nodeToContextMap.get(connection.getNode()).remove(contextId)); @@ -363,7 +367,7 @@ public void run() { for (int i = 0; i < results.getNumShards(); i++) { TestSearchPhaseResult result = results.getAtomicArray().get(i); assertEquals(result.node.getId(), result.getSearchShardTarget().getNodeId()); - sendReleaseSearchContext(result.getContextId(), new MockConnection(result.node), OriginalIndices.NONE); + sendReleaseSearchContext(result.getContextId(), new MockConnection(result.node)); } responseListener.onResponse(testResponse); if (latchTriggered.compareAndSet(false, true) == false) { @@ -421,8 +425,13 @@ public void testFanOutAndFail() throws InterruptedException { ); AtomicInteger numFreedContext = new AtomicInteger(); SearchTransportService transportService = new SearchTransportService(null, null, null) { + @Override - public void sendFreeContext(Transport.Connection connection, ShardSearchContextId contextId, OriginalIndices originalIndices) { + public void sendFreeContext( + Transport.Connection connection, + ShardSearchContextId contextId, + ActionListener listener + ) { assertNotNull(contextId); numFreedContext.incrementAndGet(); assertTrue(nodeToContextMap.containsKey(connection.getNode())); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java index 459d5573d7c12..63334bd70306f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESSingleNodeTestCase.java @@ -78,7 +78,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.elasticsearch.action.search.SearchTransportService.FREE_CONTEXT_ACTION_NAME; +import static org.elasticsearch.action.search.SearchTransportService.FREE_CONTEXT_SCROLL_ACTION_NAME; import static org.elasticsearch.cluster.coordination.ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING; import static org.elasticsearch.discovery.SettingsBasedSeedHostsProvider.DISCOVERY_SEED_HOSTS_SETTING; import static org.elasticsearch.test.NodeRoles.dataNode; @@ -482,7 +482,7 @@ protected void ensureNoInitializingShards() { */ protected void ensureAllFreeContextActionsAreConsumed() throws Exception { logger.info("--> waiting for all free_context tasks to complete within a reasonable time"); - safeGet(clusterAdmin().prepareListTasks().setActions(FREE_CONTEXT_ACTION_NAME + "*").setWaitForCompletion(true).execute()); + safeGet(clusterAdmin().prepareListTasks().setActions(FREE_CONTEXT_SCROLL_ACTION_NAME + "*").setWaitForCompletion(true).execute()); } /** From b5c1573640c7db8c462073479ac48d02ee75817d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 27 Nov 2024 10:27:20 +0100 Subject: [PATCH 006/132] AiO part 1 --- .../search/AbstractSearchAsyncAction.java | 8 +- .../action/search/AsyncSearchContext.java | 24 + .../action/search/ExpandSearchPhase.java | 4 +- .../action/search/FetchLookupFieldsPhase.java | 8 +- .../action/search/FetchSearchPhase.java | 3 +- .../action/search/RankFeaturePhase.java | 18 +- .../SearchQueryThenFetchAsyncAction.java | 632 +++++++++++++++++- .../action/search/TransportSearchAction.java | 11 +- 8 files changed, 649 insertions(+), 59 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 09fb70fb06ba4..3a0330c2fbc46 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -68,8 +68,8 @@ * The fan out and collect algorithm is traditionally used as the initial phase which can either be a query execution or collection of * distributed frequencies */ -abstract class AbstractSearchAsyncAction extends SearchPhase { - private static final float DEFAULT_INDEX_BOOST = 1.0f; +abstract class AbstractSearchAsyncAction extends SearchPhase implements AsyncSearchContext { + static final float DEFAULT_INDEX_BOOST = 1.0f; private final Logger logger; private final NamedWriteableRegistry namedWriteableRegistry; private final SearchTransportService searchTransportService; @@ -524,7 +524,7 @@ void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Excepti results.consumeShardFailure(shardIndex); } - private static boolean isTaskCancelledException(Exception e) { + static boolean isTaskCancelledException(Exception e) { return ExceptionsHelper.unwrapCausesAndSuppressed(e, ex -> ex instanceof TaskCancelledException).isPresent(); } @@ -806,7 +806,7 @@ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator s */ protected abstract SearchPhase getNextPhase(); - private static final class PendingExecutions { + static final class PendingExecutions { private final Semaphore semaphore; private final ConcurrentLinkedQueue> queue = new ConcurrentLinkedQueue<>(); diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java new file mode 100644 index 0000000000000..7b1e87b9c2303 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -0,0 +1,24 @@ +/* + * 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.action.search; + +import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.search.SearchPhaseResult; + +public interface AsyncSearchContext { + + SearchRequest getRequest(); + + void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults); + + SearchTransportService getSearchTransport(); + + SearchTask getTask(); +} diff --git a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java index 8feed2aea00b0..f33a75323b9f0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java @@ -31,11 +31,11 @@ * forwards to the next phase immediately. */ final class ExpandSearchPhase extends SearchPhase { - private final AbstractSearchAsyncAction context; + private final AsyncSearchContext context; private final SearchHits searchHits; private final Supplier nextPhase; - ExpandSearchPhase(AbstractSearchAsyncAction context, SearchHits searchHits, Supplier nextPhase) { + ExpandSearchPhase(AsyncSearchContext context, SearchHits searchHits, Supplier nextPhase) { super("expand"); this.context = context; this.searchHits = searchHits; diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java index d8671bcadf86d..e2b157b602e05 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java @@ -33,15 +33,11 @@ * @see org.elasticsearch.index.mapper.LookupRuntimeFieldType */ final class FetchLookupFieldsPhase extends SearchPhase { - private final AbstractSearchAsyncAction context; + private final AsyncSearchContext context; private final SearchResponseSections searchResponse; private final AtomicArray queryResults; - FetchLookupFieldsPhase( - AbstractSearchAsyncAction context, - SearchResponseSections searchResponse, - AtomicArray queryResults - ) { + FetchLookupFieldsPhase(AsyncSearchContext context, SearchResponseSections searchResponse, AtomicArray queryResults) { super("fetch_lookup_fields"); this.context = context; this.searchResponse = searchResponse; diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 0fbface3793a8..a9731a49d4a21 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -18,6 +18,7 @@ import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.fetch.FetchSearchResult; import org.elasticsearch.search.fetch.ShardFetchSearchRequest; +import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.rank.RankDoc; import org.elasticsearch.search.rank.RankDocShardInfo; @@ -47,7 +48,7 @@ final class FetchSearchPhase extends SearchPhase { FetchSearchPhase( SearchPhaseResults resultConsumer, AggregatedDfs aggregatedDfs, - AbstractSearchAsyncAction context, + AsyncSearchContext context, @Nullable SearchPhaseController.ReducedQueryPhase reducedQueryPhase ) { this( diff --git a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java index 199228c9f992c..6a359e659bf94 100644 --- a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java @@ -38,7 +38,7 @@ public class RankFeaturePhase extends SearchPhase { private static final Logger logger = LogManager.getLogger(RankFeaturePhase.class); - private final AbstractSearchAsyncAction context; + private final AsyncSearchContext context; final SearchPhaseResults queryPhaseResults; final SearchPhaseResults rankPhaseResults; private final AggregatedDfs aggregatedDfs; @@ -48,24 +48,16 @@ public class RankFeaturePhase extends SearchPhase { RankFeaturePhase( SearchPhaseResults queryPhaseResults, AggregatedDfs aggregatedDfs, - AbstractSearchAsyncAction context, + AsyncSearchContext context, RankFeaturePhaseRankCoordinatorContext rankFeaturePhaseRankCoordinatorContext ) { super("rank-feature"); assert rankFeaturePhaseRankCoordinatorContext != null; this.rankFeaturePhaseRankCoordinatorContext = rankFeaturePhaseRankCoordinatorContext; - if (context.getNumShards() != queryPhaseResults.getNumShards()) { - throw new IllegalStateException( - "number of shards must match the length of the query results but doesn't:" - + context.getNumShards() - + "!=" - + queryPhaseResults.getNumShards() - ); - } this.context = context; this.queryPhaseResults = queryPhaseResults; this.aggregatedDfs = aggregatedDfs; - this.rankPhaseResults = new ArraySearchPhaseResults<>(context.getNumShards()); + this.rankPhaseResults = new ArraySearchPhaseResults<>(queryPhaseResults.getNumShards()); context.addReleasable(rankPhaseResults); this.progressListener = context.getTask().getProgressListener(); } @@ -94,10 +86,10 @@ void innerRun(RankFeaturePhaseRankCoordinatorContext rankFeaturePhaseRankCoordin // to operate on the first `rank_window_size * num_shards` results and merge them appropriately. SearchPhaseController.ReducedQueryPhase reducedQueryPhase = queryPhaseResults.reduce(); ScoreDoc[] queryScoreDocs = reducedQueryPhase.sortedTopDocs().scoreDocs(); // rank_window_size - final List[] docIdsToLoad = SearchPhaseController.fillDocIdsToLoad(context.getNumShards(), queryScoreDocs); + final List[] docIdsToLoad = SearchPhaseController.fillDocIdsToLoad(queryPhaseResults.getNumShards(), queryScoreDocs); final CountedCollector rankRequestCounter = new CountedCollector<>( rankPhaseResults, - context.getNumShards(), + queryPhaseResults.getNumShards(), () -> onPhaseDone(rankFeaturePhaseRankCoordinatorContext, reducedQueryPhase), context ); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index f75b84abc2f0f..006fb9457edde 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -11,27 +11,95 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.search.TopFieldDocs; +import org.apache.lucene.util.CollectionUtil; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.NoShardAvailableActionException; +import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.builder.PointInTimeBuilder; +import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; +import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.transport.Transport; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; +import java.util.function.Supplier; +import java.util.stream.Collectors; +import static org.elasticsearch.action.search.AbstractSearchAsyncAction.DEFAULT_INDEX_BOOST; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; +import static org.elasticsearch.core.Strings.format; -class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction { +class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearchContext { + + private final Logger logger; + private final NamedWriteableRegistry namedWriteableRegistry; + private final SearchTransportService searchTransportService; + private final Executor executor; + private final ActionListener listener; + private final SearchRequest request; + + /** + * Used by subclasses to resolve node ids to DiscoveryNodes. + **/ + private final BiFunction nodeIdToConnection; + private final SearchTask task; + protected final SearchPhaseResults results; + private final long clusterStateVersion; + private final TransportVersion minTransportVersion; + private final Map aliasFilter; + private final Map concreteIndexBoosts; + private final SetOnce> shardFailures = new SetOnce<>(); + private final Object shardFailuresMutex = new Object(); + private final AtomicBoolean hasShardResponse = new AtomicBoolean(false); + private final AtomicInteger successfulOps = new AtomicInteger(); + private final AtomicInteger skippedOps = new AtomicInteger(); + private final TransportSearchAction.SearchTimeProvider timeProvider; + private final SearchResponse.Clusters clusters; + + protected final GroupShardsIterator toSkipShardsIts; + protected final GroupShardsIterator shardsIts; + private final SearchShardIterator[] shardIterators; + private final Map shardIndexMap; + private final int expectedTotalOps; + private final AtomicInteger totalOps = new AtomicInteger(); + private final int maxConcurrentRequestsPerNode; + private final Map pendingExecutionsPerNode = new ConcurrentHashMap<>(); + private final boolean throttleConcurrentRequests; + private final AtomicBoolean requestCancelled = new AtomicBoolean(); + + // protected for tests + protected final List releasables = new ArrayList<>(); private final SearchProgressListener progressListener; @@ -59,25 +127,57 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction toSkipIterators = new ArrayList<>(); + final List iterators = new ArrayList<>(); + for (final SearchShardIterator iterator : shardsIts) { + if (iterator.skip()) { + toSkipIterators.add(iterator); + } else { + iterators.add(iterator); + } + } + this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); + this.shardsIts = new GroupShardsIterator<>(iterators); + + // we compute the shard index based on the natural order of the shards + // that participate in the search request. This means that this number is + // consistent between two requests that target the same shards. + Map shardMap = new HashMap<>(); + List searchIterators = new ArrayList<>(iterators); + CollectionUtil.timSort(searchIterators); + for (int i = 0; i < searchIterators.size(); i++) { + shardMap.put(searchIterators.get(i), i); + } + this.shardIndexMap = Collections.unmodifiableMap(shardMap); + this.shardIterators = searchIterators.toArray(SearchShardIterator[]::new); + + // we need to add 1 for non active partition, since we count it in the total. This means for each shard in the iterator we sum up + // it's number of active shards but use 1 as the default if no replica of a shard is active at this point. + // on a per shards level we use shardIt.remaining() to increment the totalOps pointer but add 1 for the current shard result + // we process hence we add one for the non active partition here. + this.expectedTotalOps = shardsIts.totalSizeWith1ForEmpty(); + this.maxConcurrentRequestsPerNode = request.getMaxConcurrentShardRequests(); + // in the case were we have less shards than maxConcurrentRequestsPerNode we don't need to throttle + this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size(); + this.timeProvider = timeProvider; + this.logger = logger; + this.searchTransportService = searchTransportService; + this.executor = executor; + this.request = request; + this.task = task; + this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); + this.nodeIdToConnection = nodeIdToConnection; + this.concreteIndexBoosts = concreteIndexBoosts; + this.clusterStateVersion = clusterState.version(); + this.minTransportVersion = clusterState.getMinTransportVersion(); + this.aliasFilter = aliasFilter; + this.results = resultConsumer; + // register the release of the query consumer to free up the circuit breaker memory + // at the end of the search + releasables.add(resultConsumer); + this.clusters = clusters; this.topDocsSize = getTopDocsSize(request); this.trackTotalHitsUpTo = request.resolveTrackTotalHitsUpTo(); this.progressListener = task.getProgressListener(); @@ -89,26 +189,141 @@ class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction(0) + ); + return; + } + executePhase(this); + } + + @Override + public SearchRequest getRequest() { + return request; + } + + /** + * Builds and sends the final search response back to the user. + * + * @param internalSearchResponse the internal search response + * @param queryResults the results of the query phase + */ + public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { + ShardSearchFailure[] failures = buildShardFailures(); + Boolean allowPartialResults = request.allowPartialSearchResults(); + assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; + if (allowPartialResults == false && failures.length > 0) { + raisePhaseFailure(new SearchPhaseExecutionException("", "Shard failures", null, failures)); + } else { + final String scrollId = request.scroll() != null ? TransportSearchHelper.buildScrollId(queryResults) : null; + final BytesReference searchContextId; + if (request.source() != null + && request.source().pointInTimeBuilder() != null + && request.source().pointInTimeBuilder().singleSession() == false) { + searchContextId = request.source().pointInTimeBuilder().getEncodedId(); + } else { + searchContextId = null; + } + ActionListener.respondAndRelease(listener, buildSearchResponse(internalSearchResponse, failures, scrollId, searchContextId)); + } + } + + @Override + public SearchTransportService getSearchTransport() { + return null; + } + + @Override + public SearchTask getTask() { + return null; + } + + private SearchResponse buildSearchResponse( + SearchResponseSections internalSearchResponse, + ShardSearchFailure[] failures, + String scrollId, + BytesReference searchContextId + ) { + int numSuccess = successfulOps.get(); + int numFailures = failures.length; + assert numSuccess + numFailures == results.getNumShards() + : "numSuccess(" + numSuccess + ") + numFailures(" + numFailures + ") != totalShards(" + results.getNumShards() + ")"; + return new SearchResponse( + internalSearchResponse, + scrollId, + results.getNumShards(), + numSuccess, + skippedOps.get(), + timeProvider.buildTookInMillis(), + failures, + clusters, + searchContextId + ); + } + + /** + * Builds an request for the initial search phase. + * + * @param shardIt the target {@link SearchShardIterator} + * @param shardIndex the index of the shard that is used in the coordinator node to + * tiebreak results with identical sort values + */ + protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shardIt, int shardIndex) { + AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); + assert filter != null; + float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); + ShardSearchRequest shardRequest = new ShardSearchRequest( + shardIt.getOriginalIndices(), + request, + shardIt.shardId(), + shardIndex, + results.getNumShards(), + filter, + indexBoost, + timeProvider.absoluteStartMillis(), + shardIt.getClusterAlias(), + shardIt.getSearchContextId(), + shardIt.getSearchContextKeepAlive() + ); + // if we already received a search result we can inform the shard that it + // can return a null response if the request rewrites to match none rather + // than creating an empty response in the search thread pool. + // Note that, we have to disable this shortcut for queries that create a context (scroll and search context). + shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && shardRequest.scroll() == null); + return shardRequest; + } + protected void executePhaseOnShard( final SearchShardIterator shardIt, final Transport.Connection connection, final SearchActionListener listener ) { - ShardSearchRequest request = rewriteShardSearchRequest(super.buildShardSearchRequest(shardIt, listener.requestIndex)); - getSearchTransport().sendExecuteQuery(connection, request, getTask(), listener); + ShardSearchRequest request = rewriteShardSearchRequest(buildShardSearchRequest(shardIt, listener.requestIndex)); + searchTransportService.sendExecuteQuery(connection, request, task, listener); } - @Override protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) { progressListener.notifyQueryFailure(shardIndex, shardTarget, exc); } - @Override protected void onShardResult(SearchPhaseResult result, SearchShardIterator shardIt) { QuerySearchResult queryResult = result.queryResult(); if (queryResult.isNull() == false // disable sort optims for scroll requests because they keep track of the last bottom doc locally (per shard) - && getRequest().scroll() == null + && request.scroll() == null // top docs are already consumed if the query was cancelled or in error. && queryResult.hasConsumedTopDocs() == false && queryResult.topDocs() != null @@ -123,12 +338,18 @@ && getRequest().scroll() == null } bottomSortCollector.consumeTopDocs(topDocs, queryResult.sortValueFormats()); } - super.onShardResult(result, shardIt); + assert result.getShardIndex() != -1 : "shard index is not set"; + assert result.getSearchShardTarget() != null : "search shard target must not be null"; + hasShardResponse.set(true); + if (logger.isTraceEnabled()) { + logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); + } + results.consumeResult(result, () -> onShardResultConsumed(result, shardIt)); } static SearchPhase nextPhase( Client client, - AbstractSearchAsyncAction context, + AsyncSearchContext context, SearchPhaseResults queryResults, AggregatedDfs aggregatedDfs ) { @@ -139,11 +360,14 @@ static SearchPhase nextPhase( return new RankFeaturePhase(queryResults, aggregatedDfs, context, rankFeaturePhaseCoordCtx); } - @Override protected SearchPhase getNextPhase() { return nextPhase(client, this, results, null); } + public OriginalIndices getOriginalIndices(int shardIndex) { + return shardIterators[shardIndex].getOriginalIndices(); + } + private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) { if (bottomSortCollector == null) { return request; @@ -160,4 +384,356 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) } return request; } + + @Override + public void run() throws IOException { + for (final SearchShardIterator iterator : toSkipShardsIts) { + assert iterator.skip(); + skipShard(iterator); + } + if (shardsIts.size() > 0) { + doCheckNoMissingShards(getName(), request, shardsIts); + for (int i = 0; i < shardsIts.size(); i++) { + final SearchShardIterator shardRoutings = shardsIts.get(i); + assert shardRoutings.skip() == false; + assert shardIndexMap.containsKey(shardRoutings); + int shardIndex = shardIndexMap.get(shardRoutings); + final SearchShardTarget routing = shardRoutings.nextOrNull(); + if (routing == null) { + failOnUnavailable(shardIndex, shardRoutings); + } else { + performPhaseOnShard(shardIndex, shardRoutings, routing); + } + } + } + } + + private void failOnUnavailable(int shardIndex, SearchShardIterator shardIt) { + SearchShardTarget unassignedShard = new SearchShardTarget(null, shardIt.shardId(), shardIt.getClusterAlias()); + onShardFailure(shardIndex, unassignedShard, shardIt, new NoShardAvailableActionException(shardIt.shardId())); + } + + protected void notifyListShards( + SearchProgressListener progressListener, + SearchResponse.Clusters clusters, + SearchSourceBuilder sourceBuilder + ) { + progressListener.notifyListShards( + SearchProgressListener.buildSearchShards(this.shardsIts), + SearchProgressListener.buildSearchShards(toSkipShardsIts), + clusters, + sourceBuilder == null || sourceBuilder.size() > 0, + timeProvider + ); + } + + protected void performPhaseOnShard(final int shardIndex, final SearchShardIterator shardIt, final SearchShardTarget shard) { + if (throttleConcurrentRequests) { + var pendingExecutions = pendingExecutionsPerNode.computeIfAbsent( + shard.getNodeId(), + n -> new AbstractSearchAsyncAction.PendingExecutions(maxConcurrentRequestsPerNode) + ); + pendingExecutions.submit(l -> doPerformPhaseOnShard(shardIndex, shardIt, shard, l)); + } else { + doPerformPhaseOnShard(shardIndex, shardIt, shard, () -> {}); + } + } + + private void doPerformPhaseOnShard(int shardIndex, SearchShardIterator shardIt, SearchShardTarget shard, Releasable releasable) { + var shardListener = new SearchActionListener<>(shard, shardIndex) { + @Override + public void innerOnResponse(SearchPhaseResult result) { + try { + releasable.close(); + onShardResult(result, shardIt); + } catch (Exception exc) { + onShardFailure(shardIndex, shard, shardIt, exc); + } + } + + @Override + public void onFailure(Exception e) { + releasable.close(); + onShardFailure(shardIndex, shard, shardIt, e); + } + }; + final Transport.Connection connection; + try { + connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); + } catch (Exception e) { + shardListener.onFailure(e); + return; + } + executePhaseOnShard(shardIt, connection, shardListener); + } + + public final Transport.Connection getConnection(String clusterAlias, String nodeId) { + return nodeIdToConnection.apply(clusterAlias, nodeId); + } + + private void onShardFailure(final int shardIndex, SearchShardTarget shard, final SearchShardIterator shardIt, Exception e) { + // we always add the shard failure for a specific shard instance + // we do make sure to clean it on a successful response from a shard + onShardFailure(shardIndex, shard, e); + final SearchShardTarget nextShard = shardIt.nextOrNull(); + final boolean lastShard = nextShard == null; + logger.debug(() -> format("%s: Failed to execute [%s] lastShard [%s]", shard, request, lastShard), e); + if (lastShard) { + if (request.allowPartialSearchResults() == false) { + if (requestCancelled.compareAndSet(false, true)) { + try { + searchTransportService.cancelSearchTask(task, "partial results are not allowed and at least one shard has failed"); + } catch (Exception cancelFailure) { + logger.debug("Failed to cancel search request", cancelFailure); + } + } + } + onShardGroupFailure(shardIndex, shard, e); + } + final int totalOps = this.totalOps.incrementAndGet(); + if (totalOps == expectedTotalOps) { + onPhaseDone(); + } else if (totalOps > expectedTotalOps) { + throw new AssertionError( + "unexpected higher total ops [" + totalOps + "] compared to expected [" + expectedTotalOps + "]", + new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) + ); + } else { + if (lastShard == false) { + performPhaseOnShard(shardIndex, shardIt, nextShard); + } + } + } + + /** + * Executed once for every failed shard level request. This method is invoked before the next replica is tried for the given + * shard target. + * @param shardIndex the internal index for this shard. Each shard has an index / ordinal assigned that is used to reference + * it's results + * @param shardTarget the shard target for this failure + * @param e the failure reason + */ + void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Exception e) { + if (TransportActions.isShardNotAvailableException(e)) { + // Groups shard not available exceptions under a generic exception that returns a SERVICE_UNAVAILABLE(503) + // temporary error. + e = NoShardAvailableActionException.forOnShardFailureWrapper(e.getMessage()); + } + // we don't aggregate shard on failures due to the internal cancellation, + // but do keep the header counts right + if ((requestCancelled.get() && AbstractSearchAsyncAction.isTaskCancelledException(e)) == false) { + AtomicArray shardFailures = this.shardFailures.get(); + // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures) + if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally + synchronized (shardFailuresMutex) { + shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it? + if (shardFailures == null) { // still null so we are the first and create a new instance + shardFailures = new AtomicArray<>(results.getNumShards()); + this.shardFailures.set(shardFailures); + } + } + } + ShardSearchFailure failure = shardFailures.get(shardIndex); + if (failure == null) { + shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); + } else { + // the failure is already present, try and not override it with an exception that is less meaningless + // for example, getting illegal shard state + if (TransportActions.isReadOverrideException(e) && (e instanceof SearchContextMissingException == false)) { + shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); + } + } + + if (results.hasResult(shardIndex)) { + assert failure == null : "shard failed before but shouldn't: " + failure; + successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter + } + } + results.consumeShardFailure(shardIndex); + } + + void skipShard(SearchShardIterator iterator) { + successfulOps.incrementAndGet(); + skippedOps.incrementAndGet(); + assert iterator.skip(); + successfulShardExecution(iterator); + } + + private void successfulShardExecution(SearchShardIterator shardsIt) { + final int remainingOpsOnIterator; + if (shardsIt.skip()) { + // It's possible that we're skipping a shard that's unavailable + // but its range was available in the IndexMetadata, in that + // case the shardsIt.remaining() would be 0, expectedTotalOps + // accounts for unavailable shards too. + remainingOpsOnIterator = Math.max(shardsIt.remaining(), 1); + } else { + remainingOpsOnIterator = shardsIt.remaining() + 1; + } + final int xTotalOps = totalOps.addAndGet(remainingOpsOnIterator); + if (xTotalOps == expectedTotalOps) { + onPhaseDone(); + } else if (xTotalOps > expectedTotalOps) { + throw new AssertionError( + "unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]", + new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) + ); + } + } + + final void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() + executeNextPhase(this, this::getNextPhase); + } + + protected void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { + /* This is the main search phase transition where we move to the next phase. If all shards + * failed or if there was a failure and partial results are not allowed, then we immediately + * fail. Otherwise we continue to the next phase. + */ + ShardOperationFailedException[] shardSearchFailures = buildShardFailures(); + if (shardSearchFailures.length == results.getNumShards()) { + shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); + Throwable cause = shardSearchFailures.length == 0 + ? null + : ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + logger.debug(() -> "All shards failed for phase: [" + currentPhase.getName() + "]", cause); + onPhaseFailure(currentPhase, "all shards failed", cause); + } else { + Boolean allowPartialResults = request.allowPartialSearchResults(); + assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; + if (allowPartialResults == false && successfulOps.get() != results.getNumShards()) { + // check if there are actual failures in the atomic array since + // successful retries can reset the failures to null + if (shardSearchFailures.length > 0) { + if (logger.isDebugEnabled()) { + int numShardFailures = shardSearchFailures.length; + shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); + Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase.getName()), cause); + } + onPhaseFailure(currentPhase, "Partial shards failure", null); + } else { + int discrepancy = results.getNumShards() - successfulOps.get(); + assert discrepancy > 0 : "discrepancy: " + discrepancy; + if (logger.isDebugEnabled()) { + logger.debug( + "Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})", + discrepancy, + successfulOps.get(), + skippedOps.get(), + results.getNumShards(), + currentPhase.getName() + ); + } + onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); + } + return; + } + var nextPhase = nextPhaseSupplier.get(); + if (logger.isTraceEnabled()) { + final String resultsFrom = results.getSuccessfulResults() + .map(r -> r.getSearchShardTarget().toString()) + .collect(Collectors.joining(",")); + logger.trace( + "[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})", + currentPhase.getName(), + nextPhase.getName(), + resultsFrom, + clusterStateVersion + ); + } + executePhase(nextPhase); + } + } + + private void executePhase(SearchPhase phase) { + try { + phase.run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, phase.getName()), e); + } + onPhaseFailure(phase, "", e); + } + } + + /** + * This method will communicate a fatal phase failure back to the user. In contrast to a shard failure + * will this method immediately fail the search request and return the failure to the issuer of the request + * @param phase the phase that failed + * @param msg an optional message + * @param cause the cause of the phase failure + */ + public void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { + raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); + } + + /** + * This method should be called if a search phase failed to ensure all relevant reader contexts are released. + * This method will also notify the listener and sends back a failure to the user. + * + * @param exception the exception explaining or causing the phase failure + */ + private void raisePhaseFailure(SearchPhaseExecutionException exception) { + results.getSuccessfulResults().forEach((entry) -> { + // Do not release search contexts that are part of the point in time + if (entry.getContextId() != null && isPartOfPointInTime(entry.getContextId()) == false) { + try { + SearchShardTarget searchShardTarget = entry.getSearchShardTarget(); + Transport.Connection connection = getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId()); + sendReleaseSearchContext(entry.getContextId(), connection, getOriginalIndices(entry.getShardIndex())); + } catch (Exception inner) { + inner.addSuppressed(exception); + logger.trace("failed to release context", inner); + } + } + }); + listener.onFailure(exception); + } + + void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection, OriginalIndices originalIndices) { + assert isPartOfPointInTime(contextId) == false : "Must not release point in time context [" + contextId + "]"; + if (connection != null) { + searchTransportService.sendFreeContext(connection, contextId, originalIndices); + } + } + + public boolean isPartOfPointInTime(ShardSearchContextId contextId) { + final PointInTimeBuilder pointInTimeBuilder = request.pointInTimeBuilder(); + if (pointInTimeBuilder != null) { + return request.pointInTimeBuilder().getSearchContextId(namedWriteableRegistry).contains(contextId); + } else { + return false; + } + } + + private ShardSearchFailure[] buildShardFailures() { + AtomicArray shardFailures = this.shardFailures.get(); + if (shardFailures == null) { + return ShardSearchFailure.EMPTY_ARRAY; + } + List entries = shardFailures.asList(); + ShardSearchFailure[] failures = new ShardSearchFailure[entries.size()]; + for (int i = 0; i < failures.length; i++) { + failures[i] = entries.get(i); + } + return failures; + } + + private void onShardResultConsumed(SearchPhaseResult result, SearchShardIterator shardIt) { + successfulOps.incrementAndGet(); + // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level + // so its ok concurrency wise to miss potentially the shard failures being created because of another failure + // in the #addShardFailure, because by definition, it will happen on *another* shardIndex + AtomicArray shardFailures = this.shardFailures.get(); + if (shardFailures != null) { + shardFailures.set(result.getShardIndex(), null); + } + // we need to increment successful ops first before we compare the exit condition otherwise if we + // are fast we could concurrently update totalOps but then preempt one of the threads which can + // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc. + // increment all the "future" shards to update the total ops since we some may work and some may not... + // and when that happens, we break on total ops, so we must maintain them + successfulShardExecution(shardIt); + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index cfdb8b4ed4191..cc90b6a422d80 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -1506,9 +1506,8 @@ public void runNewSearchPhase( ); boolean success = false; try { - final AbstractSearchAsyncAction searchPhase; if (searchRequest.searchType() == DFS_QUERY_THEN_FETCH) { - searchPhase = new SearchDfsQueryThenFetchAsyncAction( + var searchPhase = new SearchDfsQueryThenFetchAsyncAction( logger, namedWriteableRegistry, searchTransportService, @@ -1526,9 +1525,11 @@ public void runNewSearchPhase( clusters, client ); + success = true; + searchPhase.start(); } else { assert searchRequest.searchType() == QUERY_THEN_FETCH : searchRequest.searchType(); - searchPhase = new SearchQueryThenFetchAsyncAction( + var searchPhase = new SearchQueryThenFetchAsyncAction( logger, namedWriteableRegistry, searchTransportService, @@ -1546,9 +1547,9 @@ public void runNewSearchPhase( clusters, client ); + success = true; + searchPhase.start(); } - success = true; - searchPhase.start(); } finally { if (success == false) { queryResultConsumer.close(); From 301d7ec05782b103c1aa4d284cd5bb5acbed90d0 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 28 Nov 2024 09:59:27 +0100 Subject: [PATCH 007/132] bck --- .../action/search/FetchSearchPhase.java | 18 +++++------------- 1 file changed, 5 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index a9731a49d4a21..eadaf19fda3ce 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -8,6 +8,7 @@ */ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.search.ScoreDoc; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -18,7 +19,6 @@ import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.fetch.FetchSearchResult; import org.elasticsearch.search.fetch.ShardFetchSearchRequest; -import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.rank.RankDoc; import org.elasticsearch.search.rank.RankDocShardInfo; @@ -35,10 +35,11 @@ * Then it reaches out to all relevant shards to fetch the topN hits. */ final class FetchSearchPhase extends SearchPhase { + private static final Logger logger = LogManager.getLogger(FetchSearchPhase.class); + private final AtomicArray searchPhaseShardResults; private final BiFunction, SearchPhase> nextPhaseFactory; - private final AbstractSearchAsyncAction context; - private final Logger logger; + private final AsyncSearchContext context; private final SearchProgressListener progressListener; private final AggregatedDfs aggregatedDfs; @Nullable @@ -67,24 +68,15 @@ final class FetchSearchPhase extends SearchPhase { FetchSearchPhase( SearchPhaseResults resultConsumer, AggregatedDfs aggregatedDfs, - AbstractSearchAsyncAction context, + AsyncSearchContext context, @Nullable SearchPhaseController.ReducedQueryPhase reducedQueryPhase, BiFunction, SearchPhase> nextPhaseFactory ) { super("fetch"); - if (context.getNumShards() != resultConsumer.getNumShards()) { - throw new IllegalStateException( - "number of shards must match the length of the query results but doesn't:" - + context.getNumShards() - + "!=" - + resultConsumer.getNumShards() - ); - } this.searchPhaseShardResults = resultConsumer.getAtomicArray(); this.aggregatedDfs = aggregatedDfs; this.nextPhaseFactory = nextPhaseFactory; this.context = context; - this.logger = context.getLogger(); this.progressListener = context.getTask().getProgressListener(); this.reducedQueryPhase = reducedQueryPhase; this.resultConsumer = reducedQueryPhase == null ? resultConsumer : null; From bdaef37a2202cf6f65ff5cfe7689d2d07a8397cf Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 28 Nov 2024 10:51:22 +0100 Subject: [PATCH 008/132] fixup node ctor --- .../action/search/AbstractSearchAsyncAction.java | 3 ++- .../action/search/AsyncSearchContext.java | 10 ++++++++++ .../action/search/CountedCollector.java | 4 ++-- .../org/elasticsearch/action/search/SearchPhase.java | 9 +++++++-- .../search/SearchQueryThenFetchAsyncAction.java | 12 +++++++++++- 5 files changed, 32 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 3a0330c2fbc46..f862028315c9a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -485,7 +485,8 @@ protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget * @param shardTarget the shard target for this failure * @param e the failure reason */ - void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Exception e) { + @Override + public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Exception e) { if (TransportActions.isShardNotAvailableException(e)) { // Groups shard not available exceptions under a generic exception that returns a SERVICE_UNAVAILABLE(503) // temporary error. diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 7b1e87b9c2303..2bbedd4c01b1a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -10,7 +10,9 @@ package org.elasticsearch.action.search; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.core.Releasable; import org.elasticsearch.search.SearchPhaseResult; +import org.elasticsearch.search.SearchShardTarget; public interface AsyncSearchContext { @@ -21,4 +23,12 @@ public interface AsyncSearchContext { SearchTransportService getSearchTransport(); SearchTask getTask(); + + void onPhaseFailure(SearchPhase phase, String msg, Throwable cause); + + void addReleasable(Releasable releasable); + + void execute(Runnable command); + + void onShardFailure(int shardIndex, SearchShardTarget shard, Exception e); } diff --git a/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java b/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java index 3d15e11a19d31..45b8b1804f735 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java +++ b/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java @@ -22,9 +22,9 @@ final class CountedCollector { private final SearchPhaseResults resultConsumer; private final CountDown counter; private final Runnable onFinish; - private final AbstractSearchAsyncAction context; + private final AsyncSearchContext context; - CountedCollector(SearchPhaseResults resultConsumer, int expectedOps, Runnable onFinish, AbstractSearchAsyncAction context) { + CountedCollector(SearchPhaseResults resultConsumer, int expectedOps, Runnable onFinish, AsyncSearchContext context) { this.resultConsumer = resultConsumer; this.counter = new CountDown(expectedOps); this.onFinish = onFinish; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index 986f7210c0d1b..63728ded71af4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -8,6 +8,8 @@ */ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.core.CheckedRunnable; import org.elasticsearch.search.SearchPhaseResult; @@ -22,6 +24,9 @@ * Base class for all individual search phases like collecting distributed frequencies, fetching documents, querying shards. */ abstract class SearchPhase implements CheckedRunnable { + + private static final Logger logger = LogManager.getLogger(SearchPhase.class); + private final String name; protected SearchPhase(String name) { @@ -79,7 +84,7 @@ protected static void doCheckNoMissingShards( /** * Releases shard targets that are not used in the docsIdsToLoad. */ - protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPhaseResult, AbstractSearchAsyncAction context) { + protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPhaseResult, AsyncSearchContext context) { // we only release search context that we did not fetch from, if we are not scrolling // or using a PIT and if it has at least one hit that didn't make it to the global topDocs if (searchPhaseResult == null) { @@ -94,7 +99,7 @@ protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPha && context.getRequest().scroll() == null && (context.isPartOfPointInTime(phaseResult.getContextId()) == false)) { try { - context.getLogger().trace("trying to release search context [{}]", phaseResult.getContextId()); + logger.trace("trying to release search context [{}]", phaseResult.getContextId()); SearchShardTarget shardTarget = phaseResult.getSearchShardTarget(); Transport.Connection connection = context.getConnection(shardTarget.getClusterAlias(), shardTarget.getNodeId()); context.sendReleaseSearchContext( diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 006fb9457edde..6823755589b1e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -513,7 +513,7 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final * @param shardTarget the shard target for this failure * @param e the failure reason */ - void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Exception e) { + public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, Exception e) { if (TransportActions.isShardNotAvailableException(e)) { // Groups shard not available exceptions under a generic exception that returns a SERVICE_UNAVAILABLE(503) // temporary error. @@ -668,6 +668,16 @@ public void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); } + @Override + public void addReleasable(Releasable releasable) { + releasables.add(releasable); + } + + @Override + public void execute(Runnable command) { + executor.execute(command); + } + /** * This method should be called if a search phase failed to ensure all relevant reader contexts are released. * This method will also notify the listener and sends back a failure to the user. From c78fa2dd34dbbf5660b7aa2d71fd26d821fc4a43 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 28 Nov 2024 10:54:19 +0100 Subject: [PATCH 009/132] fixup node ctor --- .../org/elasticsearch/action/search/AsyncSearchContext.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 2bbedd4c01b1a..642237c825b4f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -13,6 +13,7 @@ import org.elasticsearch.core.Releasable; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.transport.Transport; public interface AsyncSearchContext { @@ -31,4 +32,6 @@ public interface AsyncSearchContext { void execute(Runnable command); void onShardFailure(int shardIndex, SearchShardTarget shard, Exception e); + + Transport.Connection getConnection(String clusterAlias, String nodeId); } From 905adcdde7be1c77d06d4b5ad782bb55bf6dc476 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 28 Nov 2024 14:02:08 +0100 Subject: [PATCH 010/132] bck --- .../action/search/AbstractSearchAsyncAction.java | 11 ++++++++++- .../action/search/AsyncSearchContext.java | 6 ++++++ .../elasticsearch/action/search/FetchSearchPhase.java | 2 +- .../org/elasticsearch/action/search/SearchPhase.java | 4 ++-- .../search/SearchQueryThenFetchAsyncAction.java | 6 +++--- 5 files changed, 22 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 3854559e7d70e..711dcf2e0f245 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -623,6 +623,14 @@ public OriginalIndices getOriginalIndices(int shardIndex) { * We should not release search contexts that belong to the point in time during or after searches. */ public boolean isPartOfPointInTime(ShardSearchContextId contextId) { + return isPartOfPIT(namedWriteableRegistry, request, contextId); + } + + public static boolean isPartOfPIT( + NamedWriteableRegistry namedWriteableRegistry, + SearchRequest request, + ShardSearchContextId contextId + ) { final PointInTimeBuilder pointInTimeBuilder = request.pointInTimeBuilder(); if (pointInTimeBuilder != null) { return request.pointInTimeBuilder().getSearchContextId(namedWriteableRegistry).contains(contextId); @@ -728,7 +736,8 @@ private void raisePhaseFailure(SearchPhaseExecutionException exception) { * @see org.elasticsearch.search.fetch.FetchSearchResult#getContextId() * */ - void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { + @Override + public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { assert isPartOfPointInTime(contextId) == false : "Must not release point in time context [" + contextId + "]"; if (connection != null) { searchTransportService.sendFreeContext(connection, contextId, ActionListener.noop()); diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 642237c825b4f..63076c4ece4f4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -9,10 +9,12 @@ package org.elasticsearch.action.search; +import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.transport.Transport; public interface AsyncSearchContext { @@ -34,4 +36,8 @@ public interface AsyncSearchContext { void onShardFailure(int shardIndex, SearchShardTarget shard, Exception e); Transport.Connection getConnection(String clusterAlias, String nodeId); + + OriginalIndices getOriginalIndices(int shardIndex); + + void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection); } diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index eadaf19fda3ce..cb89daf8340ab 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -102,7 +102,7 @@ private void innerRun() throws Exception { assert this.reducedQueryPhase == null ^ this.resultConsumer == null; // depending on whether we executed the RankFeaturePhase we may or may not have the reduced query result computed already final var reducedQueryPhase = this.reducedQueryPhase == null ? resultConsumer.reduce() : this.reducedQueryPhase; - final int numShards = context.getNumShards(); + final int numShards = searchPhaseShardResults.length(); // Usually when there is a single shard, we force the search type QUERY_THEN_FETCH. But when there's kNN, we might // still use DFS_QUERY_THEN_FETCH, which does not perform the "query and fetch" optimization during the query phase. final boolean queryAndFetchOptimization = searchPhaseShardResults.length() == 1 diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index 482d008a40b7b..8d1b9259f6f7b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -97,14 +97,14 @@ protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPha if (phaseResult != null && phaseResult.hasSearchContext() && context.getRequest().scroll() == null - && (context.isPartOfPointInTime(phaseResult.getContextId()) == false)) { + && (AbstractSearchAsyncAction.isPartOfPIT(null, context.getRequest(), phaseResult.getContextId()) == false)) { try { logger.trace("trying to release search context [{}]", phaseResult.getContextId()); SearchShardTarget shardTarget = phaseResult.getSearchShardTarget(); Transport.Connection connection = context.getConnection(shardTarget.getClusterAlias(), shardTarget.getNodeId()); context.sendReleaseSearchContext(phaseResult.getContextId(), connection); } catch (Exception e) { - context.getLogger().trace("failed to release context", e); + logger.trace("failed to release context", e); } } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 6823755589b1e..91ea4fac94e65 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -691,7 +691,7 @@ private void raisePhaseFailure(SearchPhaseExecutionException exception) { try { SearchShardTarget searchShardTarget = entry.getSearchShardTarget(); Transport.Connection connection = getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId()); - sendReleaseSearchContext(entry.getContextId(), connection, getOriginalIndices(entry.getShardIndex())); + sendReleaseSearchContext(entry.getContextId(), connection); } catch (Exception inner) { inner.addSuppressed(exception); logger.trace("failed to release context", inner); @@ -701,10 +701,10 @@ private void raisePhaseFailure(SearchPhaseExecutionException exception) { listener.onFailure(exception); } - void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection, OriginalIndices originalIndices) { + public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { assert isPartOfPointInTime(contextId) == false : "Must not release point in time context [" + contextId + "]"; if (connection != null) { - searchTransportService.sendFreeContext(connection, contextId, originalIndices); + searchTransportService.sendFreeContext(connection, contextId, ActionListener.noop()); } } From 411b85bf6792f234b2e856940a9c37c9ff65c75b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 29 Nov 2024 09:32:47 +0100 Subject: [PATCH 011/132] compiles --- .../action/search/AbstractSearchAsyncAction.java | 2 +- .../elasticsearch/action/search/AsyncSearchContext.java | 4 ++++ .../org/elasticsearch/action/search/ExpandSearchPhase.java | 2 +- .../action/search/SearchQueryThenFetchAsyncAction.java | 7 ++++--- 4 files changed, 10 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 711dcf2e0f245..d0454d23bb0e5 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -349,7 +349,7 @@ protected abstract void executePhaseOnShard( * of the next phase. If there are no successful operations in the context when this method is executed the search is aborted and * a response is returned to the user indicating that all shards have failed. */ - protected void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { + public void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { /* This is the main search phase transition where we move to the next phase. If all shards * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 63076c4ece4f4..489631f760ce8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -17,6 +17,8 @@ import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.transport.Transport; +import java.util.function.Supplier; + public interface AsyncSearchContext { SearchRequest getRequest(); @@ -40,4 +42,6 @@ public interface AsyncSearchContext { OriginalIndices getOriginalIndices(int shardIndex); void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection); + + void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier); } diff --git a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java index f33a75323b9f0..81ca45f58ae9b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java @@ -119,7 +119,7 @@ private void doRun() { } } onPhaseDone(); - }, context::onFailure)); + }, e -> context.onPhaseFailure(this, "", e))); } private static SearchSourceBuilder buildExpandSearchSourceBuilder(InnerHitBuilder options, CollapseBuilder innerCollapseBuilder) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 91ea4fac94e65..bc465a0babaf7 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -243,12 +243,12 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At @Override public SearchTransportService getSearchTransport() { - return null; + return searchTransportService; } @Override public SearchTask getTask() { - return null; + return task; } private SearchResponse buildSearchResponse( @@ -585,7 +585,8 @@ final void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() executeNextPhase(this, this::getNextPhase); } - protected void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { + @Override + public void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { /* This is the main search phase transition where we move to the next phase. If all shards * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. From 789cd61d7a2892e973cc1c78cca2dbe5c30aefd6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 29 Nov 2024 13:40:57 +0100 Subject: [PATCH 012/132] bck --- .../search/QueryPhaseResultConsumer.java | 31 +++++- .../SearchQueryThenFetchAsyncAction.java | 98 ++++++++++++++----- .../action/search/SearchTransportService.java | 16 +++ 3 files changed, 119 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 37d5065fdd031..19b91a372d149 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -16,8 +16,13 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.CircuitBreakingException; import org.elasticsearch.common.io.stream.DelayableWriteable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; @@ -27,6 +32,7 @@ import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.search.rank.context.QueryPhaseRankCoordinatorContext; +import java.io.IOException; import java.util.ArrayDeque; import java.util.ArrayList; import java.util.Arrays; @@ -526,12 +532,33 @@ private static void releaseAggs(QuerySearchResult... toConsume) { } } - private record MergeResult( + public record MergeResult( List processedShards, TopDocs reducedTopDocs, InternalAggregations reducedAggs, long estimatedSize - ) {} + ) implements Writeable { + + static MergeResult readFrom(StreamInput in) throws IOException { + return new MergeResult( + in.readCollectionAsImmutableList(i -> new SearchShard(i.readOptionalString(), new ShardId(i))), + Lucene.readTopDocs(in).topDocs, + in.readOptionalWriteable(InternalAggregations::readFrom), + in.readVLong() + ); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeCollection(processedShards, (o, s) -> { + o.writeOptionalString(s.clusterAlias()); + s.shardId().writeTo(o); + }); + Lucene.writeTopDocs(out, new TopDocsAndMaxScore(reducedTopDocs, 0.0f)); + out.writeOptionalWriteable(reducedAggs); + out.writeVLong(estimatedSize); + } + } private static class MergeTask { private final List emptyResults; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index bc465a0babaf7..5ea3403fe1f63 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -26,6 +26,9 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -41,6 +44,8 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportResponse; import java.io.IOException; import java.util.ArrayList; @@ -48,7 +53,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -93,9 +97,6 @@ class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearch private final Map shardIndexMap; private final int expectedTotalOps; private final AtomicInteger totalOps = new AtomicInteger(); - private final int maxConcurrentRequestsPerNode; - private final Map pendingExecutionsPerNode = new ConcurrentHashMap<>(); - private final boolean throttleConcurrentRequests; private final AtomicBoolean requestCancelled = new AtomicBoolean(); // protected for tests @@ -158,9 +159,6 @@ class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearch // on a per shards level we use shardIt.remaining() to increment the totalOps pointer but add 1 for the current shard result // we process hence we add one for the non active partition here. this.expectedTotalOps = shardsIts.totalSizeWith1ForEmpty(); - this.maxConcurrentRequestsPerNode = request.getMaxConcurrentShardRequests(); - // in the case were we have less shards than maxConcurrentRequestsPerNode we don't need to throttle - this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size(); this.timeProvider = timeProvider; this.logger = logger; this.searchTransportService = searchTransportService; @@ -306,13 +304,71 @@ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator s return shardRequest; } - protected void executePhaseOnShard( - final SearchShardIterator shardIt, - final Transport.Connection connection, - final SearchActionListener listener - ) { - ShardSearchRequest request = rewriteShardSearchRequest(buildShardSearchRequest(shardIt, listener.requestIndex)); - searchTransportService.sendExecuteQuery(connection, request, task, listener); + public static class NodeQueryResponse extends TransportResponse { + private final Map failedShards; + private final QueryPhaseResultConsumer.MergeResult mergeResult; + + NodeQueryResponse(StreamInput in) throws IOException { + super(in); + this.failedShards = in.readMap(StreamInput::readVInt, StreamInput::readException); + this.mergeResult = QueryPhaseResultConsumer.MergeResult.readFrom(in); + } + + NodeQueryResponse(Map failedShards, QueryPhaseResultConsumer.MergeResult mergeResult) { + this.failedShards = failedShards; + this.mergeResult = mergeResult; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeMap(failedShards, StreamOutput::writeVInt, StreamOutput::writeException); + mergeResult.writeTo(out); + } + } + + public static class NodeQueryRequest extends TransportRequest { + private final List shards; + private final SearchRequest searchRequest; + + private NodeQueryRequest(List shards, SearchRequest searchRequest) { + this.shards = shards; + this.searchRequest = searchRequest; + } + + private NodeQueryRequest(StreamInput in) throws IOException { + super(in); + this.shards = in.readCollectionAsImmutableList(ShardToQuery::readFrom); + this.searchRequest = new SearchRequest(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeCollection(shards); + searchRequest.writeTo(out); + } + } + + private record ShardToQuery(float boost, OriginalIndices originalIndices, int shardIndex, ShardSearchContextId contextId) + implements + Writeable { + + static ShardToQuery readFrom(StreamInput in) throws IOException { + return new ShardToQuery( + in.readFloat(), + OriginalIndices.readOriginalIndices(in), + in.readVInt(), + in.readOptionalWriteable(ShardSearchContextId::new) + ); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeFloat(boost); + OriginalIndices.writeOriginalIndices(originalIndices, out); + out.writeVInt(shardIndex); + out.writeOptionalWriteable(contextId); + } } protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget, Exception exc) { @@ -428,15 +484,7 @@ protected void notifyListShards( } protected void performPhaseOnShard(final int shardIndex, final SearchShardIterator shardIt, final SearchShardTarget shard) { - if (throttleConcurrentRequests) { - var pendingExecutions = pendingExecutionsPerNode.computeIfAbsent( - shard.getNodeId(), - n -> new AbstractSearchAsyncAction.PendingExecutions(maxConcurrentRequestsPerNode) - ); - pendingExecutions.submit(l -> doPerformPhaseOnShard(shardIndex, shardIt, shard, l)); - } else { - doPerformPhaseOnShard(shardIndex, shardIt, shard, () -> {}); - } + doPerformPhaseOnShard(shardIndex, shardIt, shard, () -> {}); } private void doPerformPhaseOnShard(int shardIndex, SearchShardIterator shardIt, SearchShardTarget shard, Releasable releasable) { @@ -464,9 +512,11 @@ public void onFailure(Exception e) { shardListener.onFailure(e); return; } - executePhaseOnShard(shardIt, connection, shardListener); + ShardSearchRequest request = rewriteShardSearchRequest(buildShardSearchRequest(shardIt, shardIndex)); + searchTransportService.sendExecuteQuery(connection, request, task, shardListener); } + @Override public final Transport.Connection getConnection(String clusterAlias, String nodeId) { return nodeIdToConnection.apply(clusterAlias, nodeId); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index cfc2e1bcdaf2b..79388693556ce 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -60,6 +60,7 @@ import java.io.IOException; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.Executor; @@ -176,6 +177,21 @@ public void sendExecuteDfs( ); } + void sendExecuteQuery( + Transport.Connection connection, + SearchQueryThenFetchAsyncAction.NodeQueryRequest request, + SearchTask task, + ActionListener listener + ) { + transportService.sendChildRequest( + connection, + "BULK_SEARCH_NAME", + request, + task, + new ConnectionCountingHandler<>(listener, SearchQueryThenFetchAsyncAction.NodeQueryResponse::new, connection) + ); + } + public void sendExecuteQuery( Transport.Connection connection, final ShardSearchRequest request, From a2264b8fbd09b3861fd3bd93a3ccbb00f3b4d459 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 29 Nov 2024 13:44:58 +0100 Subject: [PATCH 013/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 46 +++++++++---------- .../action/search/SearchTransportService.java | 16 +------ 2 files changed, 24 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 5ea3403fe1f63..1c231593a8c78 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -420,6 +420,7 @@ protected SearchPhase getNextPhase() { return nextPhase(client, this, results, null); } + @Override public OriginalIndices getOriginalIndices(int shardIndex) { return shardIterators[shardIndex].getOriginalIndices(); } @@ -484,36 +485,33 @@ protected void notifyListShards( } protected void performPhaseOnShard(final int shardIndex, final SearchShardIterator shardIt, final SearchShardTarget shard) { - doPerformPhaseOnShard(shardIndex, shardIt, shard, () -> {}); - } - - private void doPerformPhaseOnShard(int shardIndex, SearchShardIterator shardIt, SearchShardTarget shard, Releasable releasable) { - var shardListener = new SearchActionListener<>(shard, shardIndex) { - @Override - public void innerOnResponse(SearchPhaseResult result) { - try { - releasable.close(); - onShardResult(result, shardIt); - } catch (Exception exc) { - onShardFailure(shardIndex, shard, shardIt, exc); - } - } - - @Override - public void onFailure(Exception e) { - releasable.close(); - onShardFailure(shardIndex, shard, shardIt, e); - } - }; final Transport.Connection connection; try { connection = getConnection(shard.getClusterAlias(), shard.getNodeId()); } catch (Exception e) { - shardListener.onFailure(e); + onShardFailure(shardIndex, shard, shardIt, e); return; } - ShardSearchRequest request = rewriteShardSearchRequest(buildShardSearchRequest(shardIt, shardIndex)); - searchTransportService.sendExecuteQuery(connection, request, task, shardListener); + searchTransportService.sendExecuteQuery( + connection, + rewriteShardSearchRequest(buildShardSearchRequest(shardIt, shardIndex)), + task, + new SearchActionListener<>(shard, shardIndex) { + @Override + public void innerOnResponse(SearchPhaseResult result) { + try { + onShardResult(result, shardIt); + } catch (Exception exc) { + onShardFailure(shardIndex, shard, shardIt, exc); + } + } + + @Override + public void onFailure(Exception e) { + onShardFailure(shardIndex, shard, shardIt, e); + } + } + ); } @Override diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 79388693556ce..379e46a8d09fe 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -60,7 +60,6 @@ import java.io.IOException; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Objects; import java.util.concurrent.Executor; @@ -177,19 +176,8 @@ public void sendExecuteDfs( ); } - void sendExecuteQuery( - Transport.Connection connection, - SearchQueryThenFetchAsyncAction.NodeQueryRequest request, - SearchTask task, - ActionListener listener - ) { - transportService.sendChildRequest( - connection, - "BULK_SEARCH_NAME", - request, - task, - new ConnectionCountingHandler<>(listener, SearchQueryThenFetchAsyncAction.NodeQueryResponse::new, connection) - ); + public TransportService transportService() { + return transportService; } public void sendExecuteQuery( From 1835028d74c5c271a0b289e487e9e834f555e8ef Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 30 Nov 2024 17:47:26 +0100 Subject: [PATCH 014/132] bck --- .../action/search/FetchSearchPhase.java | 11 +- .../search/QueryPhaseResultConsumer.java | 3 +- .../action/search/SearchPhase.java | 3 - .../SearchQueryThenFetchAsyncAction.java | 104 +++++++++++++++++- .../action/search/TransportSearchAction.java | 1 + 5 files changed, 109 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index cb89daf8340ab..b43486c7d27fa 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -45,6 +45,7 @@ final class FetchSearchPhase extends SearchPhase { @Nullable private final SearchPhaseResults resultConsumer; private final SearchPhaseController.ReducedQueryPhase reducedQueryPhase; + private final int numShards; FetchSearchPhase( SearchPhaseResults resultConsumer, @@ -74,6 +75,7 @@ final class FetchSearchPhase extends SearchPhase { ) { super("fetch"); this.searchPhaseShardResults = resultConsumer.getAtomicArray(); + this.numShards = resultConsumer.getNumShards(); this.aggregatedDfs = aggregatedDfs; this.nextPhaseFactory = nextPhaseFactory; this.context = context; @@ -102,10 +104,9 @@ private void innerRun() throws Exception { assert this.reducedQueryPhase == null ^ this.resultConsumer == null; // depending on whether we executed the RankFeaturePhase we may or may not have the reduced query result computed already final var reducedQueryPhase = this.reducedQueryPhase == null ? resultConsumer.reduce() : this.reducedQueryPhase; - final int numShards = searchPhaseShardResults.length(); // Usually when there is a single shard, we force the search type QUERY_THEN_FETCH. But when there's kNN, we might // still use DFS_QUERY_THEN_FETCH, which does not perform the "query and fetch" optimization during the query phase. - final boolean queryAndFetchOptimization = searchPhaseShardResults.length() == 1 + final boolean queryAndFetchOptimization = numShards == 1 && context.getRequest().hasKnnSearch() == false && reducedQueryPhase.queryPhaseRankCoordinatorContext() == null && (context.getRequest().source() == null || context.getRequest().source().rankBuilder() == null); @@ -205,9 +206,9 @@ private void executeFetch( ) { final SearchShardTarget shardTarget = shardPhaseResult.getSearchShardTarget(); final int shardIndex = shardPhaseResult.getShardIndex(); - final ShardSearchContextId contextId = shardPhaseResult.queryResult() != null - ? shardPhaseResult.queryResult().getContextId() - : shardPhaseResult.rankFeatureResult().getContextId(); + final ShardSearchContextId contextId = (shardPhaseResult.queryResult() != null + ? shardPhaseResult.queryResult() + : shardPhaseResult.rankFeatureResult()).getContextId(); var listener = new SearchActionListener(shardTarget, shardIndex) { @Override public void innerOnResponse(FetchSearchResult result) { diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 19b91a372d149..4b553bb4823db 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.core.Nullable; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; @@ -535,7 +536,7 @@ private static void releaseAggs(QuerySearchResult... toConsume) { public record MergeResult( List processedShards, TopDocs reducedTopDocs, - InternalAggregations reducedAggs, + @Nullable InternalAggregations reducedAggs, long estimatedSize ) implements Writeable { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index 8d1b9259f6f7b..4ebcc68d389c3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -87,9 +87,6 @@ protected static void doCheckNoMissingShards( protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPhaseResult, AsyncSearchContext context) { // we only release search context that we did not fetch from, if we are not scrolling // or using a PIT and if it has at least one hit that didn't make it to the global topDocs - if (searchPhaseResult == null) { - return; - } // phaseResult.getContextId() is the same for query & rank feature results SearchPhaseResult phaseResult = searchPhaseResult.queryResult() != null ? searchPhaseResult.queryResult() diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 1c231593a8c78..392502d006fd1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -20,6 +20,7 @@ import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; @@ -29,11 +30,15 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; +import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; @@ -44,8 +49,11 @@ import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportResponseHandler; +import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.ArrayList; @@ -349,15 +357,20 @@ public void writeTo(StreamOutput out) throws IOException { } } - private record ShardToQuery(float boost, OriginalIndices originalIndices, int shardIndex, ShardSearchContextId contextId) - implements - Writeable { + private record ShardToQuery( + float boost, + OriginalIndices originalIndices, + int shardIndex, + ShardId shardId, + ShardSearchContextId contextId + ) implements Writeable { static ShardToQuery readFrom(StreamInput in) throws IOException { return new ShardToQuery( in.readFloat(), OriginalIndices.readOriginalIndices(in), in.readVInt(), + new ShardId(in), in.readOptionalWriteable(ShardSearchContextId::new) ); } @@ -367,6 +380,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeFloat(boost); OriginalIndices.writeOriginalIndices(originalIndices, out); out.writeVInt(shardIndex); + shardId.writeTo(out); out.writeOptionalWriteable(contextId); } } @@ -442,6 +456,8 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) return request; } + private static final TransportVersion BATCHED_QUERY_PHASE_VERSION = TransportVersion.current(); + @Override public void run() throws IOException { for (final SearchShardIterator iterator : toSkipShardsIts) { @@ -449,6 +465,7 @@ public void run() throws IOException { skipShard(iterator); } if (shardsIts.size() > 0) { + final Map perNodeQueries = new HashMap<>(); doCheckNoMissingShards(getName(), request, shardsIts); for (int i = 0; i < shardsIts.size(); i++) { final SearchShardIterator shardRoutings = shardsIts.get(i); @@ -459,9 +476,73 @@ public void run() throws IOException { if (routing == null) { failOnUnavailable(shardIndex, shardRoutings); } else { - performPhaseOnShard(shardIndex, shardRoutings, routing); + if (minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { + perNodeQueries.computeIfAbsent( + routing.getNodeId(), + ignored -> new NodeQueryRequest(new ArrayList<>(), request) + ).shards.add( + new ShardToQuery( + concreteIndexBoosts.getOrDefault(routing.getShardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), + getOriginalIndices(shardIndex), + shardIndex, + routing.getShardId(), + shardsIts.get(shardIndex).getSearchContextId() + ) + ); + } else { + performPhaseOnShard(shardIndex, shardRoutings, routing); + } } } + perNodeQueries.forEach((nodeId, request) -> { + try { + searchTransportService.transportService() + .sendChildRequest( + getConnection(null, nodeId), + NODE_SEARCH_ACTION_NAME, + request, + task, + new TransportResponseHandler() { + @Override + public NodeQueryResponse read(StreamInput in) throws IOException { + return new NodeQueryResponse(in); + } + + @Override + public Executor executor() { + return EsExecutors.DIRECT_EXECUTOR_SERVICE; + } + + @Override + public void handleResponse(NodeQueryResponse response) { + response.failedShards.forEach( + (sIdx, e) -> onShardFailure( + sIdx, + new SearchShardTarget(nodeId, shardIterators[sIdx].shardId(), null), + e + ) + ); + if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { + + } else { + + } + } + + @Override + public void handleException(TransportException e) { + for (ShardToQuery shard : request.shards) { + onShardFailure(shard.shardIndex, new SearchShardTarget(nodeId, shard.shardId, null), e); + } + } + } + ); + } catch (Exception e) { + for (ShardToQuery shard : request.shards) { + onShardFailure(shard.shardIndex, new SearchShardTarget(nodeId, shard.shardId, null), e); + } + } + }); } } @@ -795,4 +876,19 @@ private void onShardResultConsumed(SearchPhaseResult result, SearchShardIterator // and when that happens, we break on total ops, so we must maintain them successfulShardExecution(shardIt); } + + public static final String NODE_SEARCH_ACTION_NAME = "indices:data/read/search[query][n]"; + + public static void registerNodeSearchAction(TransportService transportService, SearchService searchService) { + transportService.registerRequestHandler( + NODE_SEARCH_ACTION_NAME, + EsExecutors.DIRECT_EXECUTOR_SERVICE, + NodeQueryRequest::new, + (request, channel, task) -> { + new ChannelActionListener<>(channel).onResponse( + new NodeQueryResponse(Map.of(), new QueryPhaseResultConsumer.MergeResult(List.of(), Lucene.EMPTY_TOP_DOCS, null, 0L)) + ); + } + ); + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index dc9a6b6c3138e..990d79532c8a8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -189,6 +189,7 @@ public TransportSearchAction( this.searchTransportService = searchTransportService; this.remoteClusterService = searchTransportService.getRemoteClusterService(); SearchTransportService.registerRequestHandler(transportService, searchService); + SearchQueryThenFetchAsyncAction.registerNodeSearchAction(transportService, searchService); this.clusterService = clusterService; this.transportService = transportService; this.searchService = searchService; From 8965344ea46ffe07f0fde2a38ede342415b027af Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 30 Nov 2024 22:46:27 +0100 Subject: [PATCH 015/132] bck --- .../action/search/SearchPhaseController.java | 30 ++++++++++++++++++- .../SearchQueryThenFetchAsyncAction.java | 16 ++++++++-- 2 files changed, 43 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index b118c2560925e..cde7cf92d1d61 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -25,6 +25,9 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.io.stream.DelayableWriteable; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.Maps; @@ -58,6 +61,7 @@ import org.elasticsearch.search.suggest.Suggest.Suggestion; import org.elasticsearch.search.suggest.completion.CompletionSuggestion; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -833,7 +837,7 @@ SearchPhaseResults newSearchPhaseResults( ); } - public static final class TopDocsStats { + public static final class TopDocsStats implements Writeable { final int trackTotalHitsUpTo; long totalHits; private TotalHits.Relation totalHitsRelation; @@ -895,6 +899,30 @@ void add(TopDocsAndMaxScore topDocs, boolean timedOut, Boolean terminatedEarly) } } } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeVInt(trackTotalHitsUpTo); + out.writeFloat(maxScore); + Lucene.writeTotalHits(out, new TotalHits(totalHits, totalHitsRelation)); + out.writeVLong(fetchHits); + out.writeFloat(maxScore); + out.writeBoolean(timedOut); + out.writeOptionalBoolean(terminatedEarly); + } + + public static TopDocsStats readFrom(StreamInput in) throws IOException { + TopDocsStats res = new TopDocsStats(in.readVInt()); + res.maxScore = in.readFloat(); + TotalHits totalHits = Lucene.readTotalHits(in); + res.totalHits = totalHits.value(); + res.totalHitsRelation = totalHits.relation(); + res.fetchHits = in.readVLong(); + res.maxScore = in.readFloat(); + res.timedOut = in.readBoolean(); + res.terminatedEarly = in.readOptionalBoolean(); + return res; + } } public record SortedTopDocs( diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 392502d006fd1..3b689e19c7990 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -315,22 +315,30 @@ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator s public static class NodeQueryResponse extends TransportResponse { private final Map failedShards; private final QueryPhaseResultConsumer.MergeResult mergeResult; + private final SearchPhaseController.TopDocsStats topDocsStats; NodeQueryResponse(StreamInput in) throws IOException { super(in); this.failedShards = in.readMap(StreamInput::readVInt, StreamInput::readException); this.mergeResult = QueryPhaseResultConsumer.MergeResult.readFrom(in); + this.topDocsStats = SearchPhaseController.TopDocsStats.readFrom(in); } - NodeQueryResponse(Map failedShards, QueryPhaseResultConsumer.MergeResult mergeResult) { + NodeQueryResponse( + Map failedShards, + QueryPhaseResultConsumer.MergeResult mergeResult, + SearchPhaseController.TopDocsStats topDocsStats + ) { this.failedShards = failedShards; this.mergeResult = mergeResult; + this.topDocsStats = topDocsStats; } @Override public void writeTo(StreamOutput out) throws IOException { out.writeMap(failedShards, StreamOutput::writeVInt, StreamOutput::writeException); mergeResult.writeTo(out); + topDocsStats.writeTo(out); } } @@ -886,7 +894,11 @@ public static void registerNodeSearchAction(TransportService transportService, S NodeQueryRequest::new, (request, channel, task) -> { new ChannelActionListener<>(channel).onResponse( - new NodeQueryResponse(Map.of(), new QueryPhaseResultConsumer.MergeResult(List.of(), Lucene.EMPTY_TOP_DOCS, null, 0L)) + new NodeQueryResponse( + Map.of(), + new QueryPhaseResultConsumer.MergeResult(List.of(), Lucene.EMPTY_TOP_DOCS, null, 0L), + new SearchPhaseController.TopDocsStats(0) + ) ); } ); From 8a87b1431ff3224eed0b0a33d81bf4dfe9e054b9 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 10:57:58 +0100 Subject: [PATCH 016/132] bck --- .../search/CountOnlyQueryPhaseResultConsumer.java | 7 +++++++ .../action/search/QueryPhaseResultConsumer.java | 4 ++++ .../search/SearchQueryThenFetchAsyncAction.java | 14 ++++++++++++-- 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java index b52d76aac4132..efbdf48e68ec9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java @@ -64,6 +64,13 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { next.run(); } + public void reduce(boolean terminatedEarly, boolean timedOut, long totalHits, TotalHits.Relation totalHitsRelation) { + relationAtomicReference.compareAndSet(TotalHits.Relation.EQUAL_TO, totalHitsRelation); + this.totalHits.add(totalHits); + this.terminatedEarly.set(terminatedEarly); + this.timedOut.set(timedOut); + } + @Override boolean hasResult(int shardIndex) { return results.contains(shardIndex); diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 4b553bb4823db..17b0537db9bce 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -157,6 +157,10 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { consume(querySearchResult, next); } + public void reduce(TopDocsStats topDocsStats, MergeResult mergeResult) { + + } + @Override public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { if (hasPendingMerges()) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 3b689e19c7990..a6c7b1ed3c0df 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -531,9 +531,19 @@ public void handleResponse(NodeQueryResponse response) { ) ); if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { - + queryPhaseResultConsumer.reduce(response.topDocsStats, response.mergeResult); } else { - + if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { + countOnlyQueryPhaseResultConsumer.reduce( + false, + false, + response.mergeResult.reducedTopDocs().totalHits.value(), + response.mergeResult.reducedTopDocs().totalHits.relation() + ); + } + } + if (successfulOps.addAndGet(request.shards.size() - response.failedShards.size()) == expectedTotalOps) { + onPhaseDone(); } } From 2e9328d2decae826b3733d13addc1b4be9d7c9cc Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 11:33:12 +0100 Subject: [PATCH 017/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 19 ++++++++++--------- 1 file changed, 10 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index a6c7b1ed3c0df..5a741288d557c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -202,9 +202,10 @@ public final void start() { if (results.getNumShards() == 0) { // no search shards to search on, bail with empty response // (it happens with search across _all with no indices around and consistent with broadcast operations) - int trackTotalHitsUpTo = request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO - : request.source().trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO - : request.source().trackTotalHitsUpTo(); + var source = request.source(); + int trackTotalHitsUpTo = source == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO + : source.trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO + : source.trackTotalHitsUpTo(); // total hits is null in the response if the tracking of total hits is disabled boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; sendSearchResponse( @@ -235,11 +236,10 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At raisePhaseFailure(new SearchPhaseExecutionException("", "Shard failures", null, failures)); } else { final String scrollId = request.scroll() != null ? TransportSearchHelper.buildScrollId(queryResults) : null; + var source = request.source(); final BytesReference searchContextId; - if (request.source() != null - && request.source().pointInTimeBuilder() != null - && request.source().pointInTimeBuilder().singleSession() == false) { - searchContextId = request.source().pointInTimeBuilder().getEncodedId(); + if (source != null && source.pointInTimeBuilder() != null && source.pointInTimeBuilder().singleSession() == false) { + searchContextId = source.pointInTimeBuilder().getEncodedId(); } else { searchContextId = null; } @@ -534,11 +534,12 @@ public void handleResponse(NodeQueryResponse response) { queryPhaseResultConsumer.reduce(response.topDocsStats, response.mergeResult); } else { if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { + var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; countOnlyQueryPhaseResultConsumer.reduce( false, false, - response.mergeResult.reducedTopDocs().totalHits.value(), - response.mergeResult.reducedTopDocs().totalHits.relation() + reducedTotalHits.value(), + reducedTotalHits.relation() ); } } From f664d176b35bcf9fe1d63d50343849f5d644339f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 11:51:36 +0100 Subject: [PATCH 018/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 21 ++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 5a741288d557c..49c2c870701d7 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -48,6 +48,7 @@ import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; @@ -61,7 +62,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.Executor; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; @@ -530,6 +533,10 @@ public void handleResponse(NodeQueryResponse response) { e ) ); + final int successfulShards = request.shards.size() - response.failedShards.size(); + if (successfulShards > 0) { + hasShardResponse.set(true); + } if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { queryPhaseResultConsumer.reduce(response.topDocsStats, response.mergeResult); } else { @@ -543,8 +550,12 @@ public void handleResponse(NodeQueryResponse response) { ); } } - if (successfulOps.addAndGet(request.shards.size() - response.failedShards.size()) == expectedTotalOps) { + final int numShards = results.getNumShards(); + final int successes = successfulOps.addAndGet(successfulShards); + if (successes == numShards) { onPhaseDone(); + } else if (successes > numShards) { + throw new AssertionError(); } } @@ -904,6 +915,14 @@ public static void registerNodeSearchAction(TransportService transportService, S EsExecutors.DIRECT_EXECUTOR_SERVICE, NodeQueryRequest::new, (request, channel, task) -> { + final BlockingQueue shards = new LinkedBlockingQueue<>(request.shards); + final int workers = Math.min( + request.shards.size(), + transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax() + ); + for (int i = 0; i < workers; i++) { + + } new ChannelActionListener<>(channel).onResponse( new NodeQueryResponse( Map.of(), From 5cac618e1801d1f34ab1ffbccb03d92f0bb033de Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 12:51:54 +0100 Subject: [PATCH 019/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 123 +++++++++++++++--- .../search/DefaultSearchContext.java | 10 +- .../elasticsearch/search/SearchService.java | 9 +- .../search/aggregations/AggregationPhase.java | 4 +- .../SignificantTermsAggregatorFactory.java | 4 +- .../internal/FilteredSearchContext.java | 6 +- .../search/internal/SearchContext.java | 6 +- .../search/internal/ShardSearchRequest.java | 2 +- .../search/rank/RankSearchContext.java | 3 +- .../search/rescore/RescorePhase.java | 3 +- .../index/SearchSlowLogTests.java | 3 +- .../search/MockSearchService.java | 10 +- .../elasticsearch/test/TestSearchContext.java | 8 +- 13 files changed, 142 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 49c2c870701d7..825de04673733 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; @@ -48,6 +49,9 @@ import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; import org.elasticsearch.search.query.QuerySearchResult; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportException; @@ -63,6 +67,7 @@ import java.util.List; import java.util.Map; import java.util.concurrent.BlockingQueue; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; @@ -286,32 +291,42 @@ private SearchResponse buildSearchResponse( /** * Builds an request for the initial search phase. * - * @param shardIt the target {@link SearchShardIterator} * @param shardIndex the index of the shard that is used in the coordinator node to * tiebreak results with identical sort values */ - protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shardIt, int shardIndex) { - AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); - assert filter != null; - float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); + private static ShardSearchRequest buildShardSearchRequest( + ShardId shardId, + String clusterAlias, + int shardIndex, + ShardSearchContextId searchContextId, + OriginalIndices originalIndices, + AliasFilter aliasFilter, + TimeValue searchContextKeepAlive, + float indexBoost, + SearchRequest searchRequest, + int totalShardCount, + long absoluteStartMillis, + boolean hasResponse + ) { + assert aliasFilter != null; ShardSearchRequest shardRequest = new ShardSearchRequest( - shardIt.getOriginalIndices(), - request, - shardIt.shardId(), + originalIndices, + searchRequest, + shardId, shardIndex, - results.getNumShards(), - filter, + totalShardCount, + aliasFilter, indexBoost, - timeProvider.absoluteStartMillis(), - shardIt.getClusterAlias(), - shardIt.getSearchContextId(), - shardIt.getSearchContextKeepAlive() + absoluteStartMillis, + clusterAlias, + searchContextId, + searchContextKeepAlive ); // if we already received a search result we can inform the shard that it // can return a null response if the request rewrites to match none rather // than creating an empty response in the search thread pool. // Note that, we have to disable this shortcut for queries that create a context (scroll and search context). - shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && shardRequest.scroll() == null); + shardRequest.canReturnNullResponseIfMatchNoDocs(hasResponse && shardRequest.scroll() == null); return shardRequest; } @@ -360,6 +375,11 @@ private NodeQueryRequest(StreamInput in) throws IOException { this.searchRequest = new SearchRequest(in); } + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new SearchShardTask(id, type, action, "", parentTaskId, headers); + } + @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); @@ -605,7 +625,22 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat } searchTransportService.sendExecuteQuery( connection, - rewriteShardSearchRequest(buildShardSearchRequest(shardIt, shardIndex)), + rewriteShardSearchRequest( + buildShardSearchRequest( + shardIt.shardId(), + shardIt.getClusterAlias(), + shardIndex, + shardIt.getSearchContextId(), + shardIt.getOriginalIndices(), + aliasFilter.get(shardIt.shardId().getIndex().getUUID()), + shardIt.getSearchContextKeepAlive(), + concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), + request, + results.getNumShards(), + timeProvider.absoluteStartMillis(), + hasShardResponse.get() + ) + ), task, new SearchActionListener<>(shard, shardIndex) { @Override @@ -920,8 +955,16 @@ public static void registerNodeSearchAction(TransportService transportService, S request.shards.size(), transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax() ); + var executor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH); + final ConcurrentHashMap failures = new ConcurrentHashMap<>(); + final AtomicInteger shardIndex = new AtomicInteger(); for (int i = 0; i < workers; i++) { - + ShardToQuery shardToQuery; + if ((shardToQuery = shards.poll()) != null) { + executor.execute( + () -> executeOne(searchService, request, (CancellableTask) task, shardToQuery, shardIndex, shards, executor) + ); + } } new ChannelActionListener<>(channel).onResponse( new NodeQueryResponse( @@ -932,5 +975,51 @@ public static void registerNodeSearchAction(TransportService transportService, S ); } ); + + } + + private static void executeOne( + SearchService searchService, + NodeQueryRequest request, + CancellableTask task, + ShardToQuery shardToQuery, + AtomicInteger shardIndex, + BlockingQueue shards, + Executor executor + ) { + final ShardSearchRequest req = buildShardSearchRequest( + shardToQuery.shardId, + "", + shardIndex.getAndIncrement(), + shardToQuery.contextId, + shardToQuery.originalIndices, + AliasFilter.EMPTY, + shardToQuery.contextId == null ? null : TimeValue.MAX_VALUE, + shardToQuery.boost, + request.searchRequest, + 2, + System.currentTimeMillis(), + false + ); + searchService.executeQueryPhase(req, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + + maybeNext(); + } + + @Override + public void onFailure(Exception e) { + + maybeNext(); + } + + private void maybeNext() { + var shardToQuery = shards.poll(); + if (shardToQuery != null) { + executor.execute(() -> executeOne(searchService, request, task, shardToQuery, shardIndex, shards, executor)); + } + } + }); } } diff --git a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java index 8ac35f7c40caa..fecb2617b9a5a 100644 --- a/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/DefaultSearchContext.java @@ -21,7 +21,6 @@ import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; import org.apache.lucene.util.NumericUtils; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.cluster.routing.IndexRouting; import org.elasticsearch.common.lucene.search.Queries; @@ -76,6 +75,7 @@ import org.elasticsearch.search.slice.SliceBuilder; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; +import org.elasticsearch.tasks.CancellableTask; import java.io.IOException; import java.io.UncheckedIOException; @@ -130,7 +130,7 @@ final class DefaultSearchContext extends SearchContext { private CollapseContext collapse; // filter for sliced scroll private SliceBuilder sliceBuilder; - private SearchShardTask task; + private CancellableTask task; private QueryPhaseRankShardContext queryPhaseRankShardContext; /** @@ -432,7 +432,7 @@ public void preProcess() { this.query = buildFilteredQuery(query); if (lowLevelCancellation) { searcher().addQueryCancellation(() -> { - final SearchShardTask task = getTask(); + final CancellableTask task = getTask(); if (task != null) { task.ensureNotCancelled(); } @@ -907,12 +907,12 @@ public void setProfilers(Profilers profilers) { } @Override - public void setTask(SearchShardTask task) { + public void setTask(CancellableTask task) { this.task = task; } @Override - public SearchShardTask getTask() { + public CancellableTask getTask() { return task; } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index e17709ed78318..eaae8bdd2ed41 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -123,6 +123,7 @@ import org.elasticsearch.search.sort.SortBuilder; import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.completion.CompletionSuggestion; +import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.telemetry.tracing.Tracer; import org.elasticsearch.threadpool.Scheduler; @@ -540,7 +541,7 @@ private void loadOrExecuteQueryPhase(final ShardSearchRequest request, final Sea } } - public void executeQueryPhase(ShardSearchRequest request, SearchShardTask task, ActionListener listener) { + public void executeQueryPhase(ShardSearchRequest request, CancellableTask task, ActionListener listener) { assert request.canReturnNullResponseIfMatchNoDocs() == false || request.numberOfShards() > 1 : "empty responses require more than one shard"; final IndexShard shard = getShard(request); @@ -686,7 +687,7 @@ private static void runAsync( * It is the responsibility of the caller to ensure that the ref count is correctly decremented * when the object is no longer needed. */ - private SearchPhaseResult executeQueryPhase(ShardSearchRequest request, SearchShardTask task) throws Exception { + private SearchPhaseResult executeQueryPhase(ShardSearchRequest request, CancellableTask task) throws Exception { final ReaderContext readerContext = createOrGetReaderContext(request); try ( Releasable scope = tracer.withScope(task); @@ -933,7 +934,7 @@ public void executeFetchPhase(ShardFetchRequest request, SearchShardTask task, A }, wrapFailureListener(listener, readerContext, markAsUsed)); } - protected void checkCancelled(SearchShardTask task) { + protected void checkCancelled(CancellableTask task) { // check cancellation as early as possible, as it avoids opening up a Lucene reader on FrozenEngine try { task.ensureNotCancelled(); @@ -1064,7 +1065,7 @@ public void openReaderContext(ShardId shardId, TimeValue keepAlive, ActionListen protected SearchContext createContext( ReaderContext readerContext, ShardSearchRequest request, - SearchShardTask task, + CancellableTask task, ResultsType resultsType, boolean includeAggregations ) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java index 8ca21db1ad9f2..a80f03cb8e731 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationPhase.java @@ -8,10 +8,10 @@ */ package org.elasticsearch.search.aggregations; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.search.aggregations.support.TimeSeriesIndexSearcher; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.query.QueryPhase; +import org.elasticsearch.tasks.CancellableTask; import java.io.IOException; import java.util.ArrayList; @@ -75,7 +75,7 @@ private static List getCancellationChecks(SearchContext context) { if (context.lowLevelCancellation()) { // This searching doesn't live beyond this phase, so we don't need to remove query cancellation cancellationChecks.add(() -> { - final SearchShardTask task = context.getTask(); + final CancellableTask task = context.getTask(); if (task != null) { task.ensureNotCancelled(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorFactory.java index 080cac9cbfb85..17f63643b83c2 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/SignificantTermsAggregatorFactory.java @@ -13,7 +13,6 @@ import org.apache.lucene.search.MatchNoDocsQuery; import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.common.logging.DeprecationCategory; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.index.query.QueryBuilder; @@ -128,7 +127,8 @@ private static SignificantTermsAggregatorSupplier bytesSupplier() { *

* Some searches that will never match can still fall through and we endup running query that will produce no results. * However even in that case we sometimes do expensive things like loading global ordinals. This method should prevent this. - * Note that if {@link org.elasticsearch.search.SearchService#executeQueryPhase(ShardSearchRequest, SearchShardTask, ActionListener)} + * Note that if {@link org.elasticsearch.search.SearchService#executeQueryPhase(ShardSearchRequest, + * org.elasticsearch.tasks.CancellableTask, ActionListener)} * always do a can match then we don't need this code here. */ static boolean matchNoDocs(AggregationContext context, Aggregator parent) { diff --git a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java index 8c4f912c5988c..5bad06d08f96b 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/FilteredSearchContext.java @@ -12,7 +12,6 @@ import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; @@ -40,6 +39,7 @@ import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; +import org.elasticsearch.tasks.CancellableTask; import java.util.List; @@ -422,12 +422,12 @@ public SearchExecutionContext getSearchExecutionContext() { } @Override - public void setTask(SearchShardTask task) { + public void setTask(CancellableTask task) { in.setTask(task); } @Override - public SearchShardTask getTask() { + public CancellableTask getTask() { return in.getTask(); } diff --git a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java index 49e41559702ea..bc00bb99bcfb4 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/SearchContext.java @@ -11,7 +11,6 @@ import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.core.Assertions; import org.elasticsearch.core.Nullable; @@ -47,6 +46,7 @@ import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; +import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.transport.LeakTracker; import java.io.IOException; @@ -84,9 +84,9 @@ public abstract class SearchContext implements Releasable { protected SearchContext() {} - public abstract void setTask(SearchShardTask task); + public abstract void setTask(CancellableTask task); - public abstract SearchShardTask getTask(); + public abstract CancellableTask getTask(); public abstract boolean isCancelled(); diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index fcfe6fbb70030..e96088561ab02 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -225,7 +225,7 @@ public ShardSearchRequest( long nowInMillis, @Nullable String clusterAlias, ShardSearchContextId readerId, - TimeValue keepAlive, + @Nullable TimeValue keepAlive, long waitForCheckpoint, TimeValue waitForCheckpointsTimeout, boolean forceSyntheticSource diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java index ad70e7d39aff8..2f09e3404d551 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java @@ -48,6 +48,7 @@ import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; +import org.elasticsearch.tasks.CancellableTask; import java.util.List; @@ -211,7 +212,7 @@ public long getRelativeTimeInMillis() { /* ---- ALL METHODS ARE UNSUPPORTED BEYOND HERE ---- */ @Override - public void setTask(SearchShardTask task) { + public void setTask(CancellableTask task) { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java b/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java index 7e3646e7689cc..a697960993994 100644 --- a/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java +++ b/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java @@ -22,6 +22,7 @@ import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.query.QueryPhase; import org.elasticsearch.search.query.SearchTimeoutException; +import org.elasticsearch.tasks.CancellableTask; import java.io.IOException; import java.util.ArrayList; @@ -126,7 +127,7 @@ static Runnable getCancellationChecks(SearchContext context) { List cancellationChecks = new ArrayList<>(); if (context.lowLevelCancellation()) { cancellationChecks.add(() -> { - final SearchShardTask task = context.getTask(); + final CancellableTask task = context.getTask(); if (task != null) { task.ensureNotCancelled(); } diff --git a/server/src/test/java/org/elasticsearch/index/SearchSlowLogTests.java b/server/src/test/java/org/elasticsearch/index/SearchSlowLogTests.java index 50e3269a6b9ba..05b0274298045 100644 --- a/server/src/test/java/org/elasticsearch/index/SearchSlowLogTests.java +++ b/server/src/test/java/org/elasticsearch/index/SearchSlowLogTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.TestSearchContext; @@ -93,7 +94,7 @@ public ShardSearchRequest request() { } @Override - public SearchShardTask getTask() { + public CancellableTask getTask() { return super.getTask(); } }; diff --git a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java index 79c61cacb58eb..42b11173a3b19 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java +++ b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java @@ -9,7 +9,6 @@ package org.elasticsearch.search; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.core.TimeValue; @@ -23,6 +22,7 @@ import org.elasticsearch.search.internal.ReaderContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchRequest; +import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.telemetry.tracing.Tracer; import org.elasticsearch.threadpool.ThreadPool; @@ -46,7 +46,7 @@ public static class TestPlugin extends Plugin {} private Consumer onCreateSearchContext = context -> {}; - private Function onCheckCancelled = Function.identity(); + private Function onCheckCancelled = Function.identity(); /** Throw an {@link AssertionError} if there are still in-flight contexts. */ public static void assertNoInFlightContext() { @@ -132,7 +132,7 @@ public void setOnCreateSearchContext(Consumer onCreateSearchConte protected SearchContext createContext( ReaderContext readerContext, ShardSearchRequest request, - SearchShardTask task, + CancellableTask task, ResultsType resultsType, boolean includeAggregations ) throws IOException { @@ -154,12 +154,12 @@ public SearchContext createSearchContext(ShardSearchRequest request, TimeValue t return searchContext; } - public void setOnCheckCancelled(Function onCheckCancelled) { + public void setOnCheckCancelled(Function onCheckCancelled) { this.onCheckCancelled = onCheckCancelled; } @Override - protected void checkCancelled(SearchShardTask task) { + protected void checkCancelled(CancellableTask task) { super.checkCancelled(onCheckCancelled.apply(task)); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java index 103cf1c15abc1..c46442485ff9e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/TestSearchContext.java @@ -11,7 +11,6 @@ import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.IndexService; @@ -49,6 +48,7 @@ import org.elasticsearch.search.rescore.RescoreContext; import org.elasticsearch.search.sort.SortAndFormats; import org.elasticsearch.search.suggest.SuggestionSearchContext; +import org.elasticsearch.tasks.CancellableTask; import java.util.Collections; import java.util.HashMap; @@ -67,7 +67,7 @@ public class TestSearchContext extends SearchContext { ParsedQuery postFilter; Query query; Float minScore; - SearchShardTask task; + CancellableTask task; SortAndFormats sort; boolean trackScores = false; int trackTotalHitsUpTo = SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO; @@ -506,12 +506,12 @@ public SearchExecutionContext getSearchExecutionContext() { } @Override - public void setTask(SearchShardTask task) { + public void setTask(CancellableTask task) { this.task = task; } @Override - public SearchShardTask getTask() { + public CancellableTask getTask() { return task; } From a85dd20f6d2ac5d26cfee22b42a28a84a3eb8154 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 13:13:12 +0100 Subject: [PATCH 020/132] bck --- .../search/QueryPhaseResultConsumer.java | 2 +- .../SearchQueryThenFetchAsyncAction.java | 80 ++++++++++++++++--- 2 files changed, 68 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 17b0537db9bce..9ebac2407faa1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -86,7 +86,7 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults runningTask = new AtomicReference<>(); private final AtomicReference failure = new AtomicReference<>(); - private final TopDocsStats topDocsStats; + public final TopDocsStats topDocsStats; private volatile MergeResult mergeResult; private volatile boolean hasPartialReduce; private volatile int numReducePhases; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 825de04673733..516e055e296f4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -10,6 +10,7 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.SetOnce; @@ -25,13 +26,14 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; @@ -958,21 +960,58 @@ public static void registerNodeSearchAction(TransportService transportService, S var executor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH); final ConcurrentHashMap failures = new ConcurrentHashMap<>(); final AtomicInteger shardIndex = new AtomicInteger(); + final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( + request.searchRequest, + executor, + new NoopCircuitBreaker(""), + new SearchPhaseController(searchService::aggReduceContextBuilder), + ((CancellableTask) task)::isCancelled, + SearchProgressListener.NOOP, + request.shards.size(), + e -> { + throw new AssertionError(e); + } + ); + final CountDown countDown = new CountDown(request.shards.size()); + final Runnable onDone = () -> { + if (countDown.countDown()) { + try { + var mergeResult = queryPhaseResultConsumer.reduce(); + new ChannelActionListener<>(channel).onResponse( + new NodeQueryResponse( + Map.of(), + new QueryPhaseResultConsumer.MergeResult( + request.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), + new TopDocs(mergeResult.totalHits(), mergeResult.sortedTopDocs().scoreDocs()), + mergeResult.aggregations(), + 0L + ), + queryPhaseResultConsumer.topDocsStats + ) + ); + } catch (Exception e) { + throw new AssertionError(e); + } + } + }; for (int i = 0; i < workers; i++) { ShardToQuery shardToQuery; if ((shardToQuery = shards.poll()) != null) { executor.execute( - () -> executeOne(searchService, request, (CancellableTask) task, shardToQuery, shardIndex, shards, executor) + () -> executeOne( + searchService, + request, + (CancellableTask) task, + shardToQuery, + shardIndex, + shards, + executor, + queryPhaseResultConsumer, + onDone + ) ); } } - new ChannelActionListener<>(channel).onResponse( - new NodeQueryResponse( - Map.of(), - new QueryPhaseResultConsumer.MergeResult(List.of(), Lucene.EMPTY_TOP_DOCS, null, 0L), - new SearchPhaseController.TopDocsStats(0) - ) - ); } ); @@ -985,11 +1024,13 @@ private static void executeOne( ShardToQuery shardToQuery, AtomicInteger shardIndex, BlockingQueue shards, - Executor executor + Executor executor, + QueryPhaseResultConsumer queryPhaseResultConsumer, + Runnable onDone ) { final ShardSearchRequest req = buildShardSearchRequest( shardToQuery.shardId, - "", + null, shardIndex.getAndIncrement(), shardToQuery.contextId, shardToQuery.originalIndices, @@ -1004,7 +1045,8 @@ private static void executeOne( searchService.executeQueryPhase(req, task, new ActionListener<>() { @Override public void onResponse(SearchPhaseResult searchPhaseResult) { - + searchPhaseResult.setShardIndex(req.shardRequestIndex()); + queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); maybeNext(); } @@ -1017,7 +1059,19 @@ public void onFailure(Exception e) { private void maybeNext() { var shardToQuery = shards.poll(); if (shardToQuery != null) { - executor.execute(() -> executeOne(searchService, request, task, shardToQuery, shardIndex, shards, executor)); + executor.execute( + () -> executeOne( + searchService, + request, + task, + shardToQuery, + shardIndex, + shards, + executor, + queryPhaseResultConsumer, + onDone + ) + ); } } }); From ff2520b04fa9a8b18244f443674d25353988da0b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 13:32:36 +0100 Subject: [PATCH 021/132] bck --- .../search/QueryPhaseResultConsumer.java | 16 ++++++++++++- .../action/search/SearchPhaseController.java | 23 +++++++++++++++++++ 2 files changed, 38 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 17b0537db9bce..23ef4ddda9f3a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.core.Nullable; +import org.elasticsearch.core.Tuple; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; @@ -157,8 +158,12 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { consume(querySearchResult, next); } - public void reduce(TopDocsStats topDocsStats, MergeResult mergeResult) { + private final List> batchedResults = new ArrayList<>(); + public void reduce(TopDocsStats topDocsStats, MergeResult mergeResult) { + synchronized (this) { + batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); + } } @Override @@ -186,6 +191,15 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { aggsList.add(DelayableWriteable.referencing(mergeResult.reducedAggs)); } } + for (Tuple batchedResult : batchedResults) { + if (topDocsList != null) { + topDocsList.add(batchedResult.v2().reducedTopDocs); + } + if (aggsList != null) { + aggsList.add(DelayableWriteable.referencing(batchedResult.v2().reducedAggs)); + } + topDocsStats.add(batchedResult.v1(), false, false); + } for (QuerySearchResult result : buffer) { topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly()); if (topDocsList != null) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index cde7cf92d1d61..e0c2d90abb351 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -877,6 +877,29 @@ TotalHits getTotalHits() { } } + void add(TopDocsStats other, boolean timedOut, Boolean terminatedEarly) { + if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED) { + totalHits += other.totalHits; + if (other.totalHitsRelation == Relation.GREATER_THAN_OR_EQUAL_TO) { + totalHitsRelation = TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO; + } + } + fetchHits += other.fetchHits; + if (Float.isNaN(other.maxScore) == false) { + maxScore = Math.max(maxScore, other.maxScore); + } + if (timedOut) { + this.timedOut = true; + } + if (terminatedEarly != null) { + if (this.terminatedEarly == null) { + this.terminatedEarly = terminatedEarly; + } else if (terminatedEarly) { + this.terminatedEarly = true; + } + } + } + void add(TopDocsAndMaxScore topDocs, boolean timedOut, Boolean terminatedEarly) { if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED) { totalHits += topDocs.topDocs.totalHits.value(); From 5dafe70419151ee62ec2a4a1b132d6b8de3a2bd5 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 14:02:56 +0100 Subject: [PATCH 022/132] bck --- .../search/SearchQueryThenFetchAsyncAction.java | 11 +++++++++-- .../java/org/elasticsearch/common/lucene/Lucene.java | 2 +- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 516e055e296f4..3b22716d61658 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -979,7 +979,7 @@ public static void registerNodeSearchAction(TransportService transportService, S var mergeResult = queryPhaseResultConsumer.reduce(); new ChannelActionListener<>(channel).onResponse( new NodeQueryResponse( - Map.of(), + Map.copyOf(failures), new QueryPhaseResultConsumer.MergeResult( request.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), new TopDocs(mergeResult.totalHits(), mergeResult.sortedTopDocs().scoreDocs()), @@ -991,6 +991,8 @@ public static void registerNodeSearchAction(TransportService transportService, S ); } catch (Exception e) { throw new AssertionError(e); + } finally { + queryPhaseResultConsumer.close(); } } }; @@ -1007,6 +1009,7 @@ public static void registerNodeSearchAction(TransportService transportService, S shards, executor, queryPhaseResultConsumer, + failures, onDone ) ); @@ -1026,6 +1029,7 @@ private static void executeOne( BlockingQueue shards, Executor executor, QueryPhaseResultConsumer queryPhaseResultConsumer, + Map failures, Runnable onDone ) { final ShardSearchRequest req = buildShardSearchRequest( @@ -1052,7 +1056,9 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { @Override public void onFailure(Exception e) { - + failures.put(req.shardRequestIndex(), e); + queryPhaseResultConsumer.consumeShardFailure(req.shardRequestIndex()); + onDone.run(); maybeNext(); } @@ -1069,6 +1075,7 @@ private void maybeNext() { shards, executor, queryPhaseResultConsumer, + failures, onDone ) ); diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index a57b8b4d23cdb..03e955edc0ed2 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -499,7 +499,7 @@ public static void writeFieldDoc(StreamOutput out, FieldDoc fieldDoc) throws IOE public static void writeScoreDoc(StreamOutput out, ScoreDoc scoreDoc) throws IOException { if (scoreDoc.getClass().equals(ScoreDoc.class) == false) { - throw new IllegalArgumentException("This method can only be used to serialize a ScoreDoc, not a " + scoreDoc.getClass()); + // throw new IllegalArgumentException("This method can only be used to serialize a ScoreDoc, not a " + scoreDoc.getClass()); } out.writeVInt(scoreDoc.doc); out.writeFloat(scoreDoc.score); From f3e377f1393a1a8e96d9b761d973196c8f6e8d1a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 15:17:39 +0100 Subject: [PATCH 023/132] bck --- .../action/search/QueryPhaseResultConsumer.java | 8 +++++--- .../action/search/SearchQueryThenFetchAsyncAction.java | 7 ++++--- .../main/java/org/elasticsearch/common/lucene/Lucene.java | 2 +- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 9ac0d3b3fe835..eeaaf8e79785c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; import org.elasticsearch.action.search.SearchPhaseController.TopDocsStats; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -198,7 +199,7 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { if (aggsList != null) { aggsList.add(DelayableWriteable.referencing(batchedResult.v2().reducedAggs)); } - topDocsStats.add(batchedResult.v1(), false, false); + topDocsStats.add(batchedResult.v1(), batchedResult.v1().timedOut, batchedResult.v1().terminatedEarly); } for (QuerySearchResult result : buffer) { topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly()); @@ -561,7 +562,7 @@ public record MergeResult( static MergeResult readFrom(StreamInput in) throws IOException { return new MergeResult( in.readCollectionAsImmutableList(i -> new SearchShard(i.readOptionalString(), new ShardId(i))), - Lucene.readTopDocs(in).topDocs, + new TopDocs(Lucene.readTotalHits(in), in.readArray(Lucene::readScoreDoc, ScoreDoc[]::new)), in.readOptionalWriteable(InternalAggregations::readFrom), in.readVLong() ); @@ -573,7 +574,8 @@ public void writeTo(StreamOutput out) throws IOException { o.writeOptionalString(s.clusterAlias()); s.shardId().writeTo(o); }); - Lucene.writeTopDocs(out, new TopDocsAndMaxScore(reducedTopDocs, 0.0f)); + Lucene.writeTotalHits(out, reducedTopDocs.totalHits); + out.writeArray(Lucene::writeScoreDoc, reducedTopDocs.scoreDocs); out.writeOptionalWriteable(reducedAggs); out.writeVLong(estimatedSize); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 3b22716d61658..89089a504289e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -573,10 +573,11 @@ public void handleResponse(NodeQueryResponse response) { } } final int numShards = results.getNumShards(); - final int successes = successfulOps.addAndGet(successfulShards); - if (successes == numShards) { + successfulOps.addAndGet(successfulShards); + final int total = totalOps.addAndGet(successfulShards + response.failedShards.size()); + if (total == numShards) { onPhaseDone(); - } else if (successes > numShards) { + } else if (total > numShards) { throw new AssertionError(); } } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 03e955edc0ed2..a57b8b4d23cdb 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -499,7 +499,7 @@ public static void writeFieldDoc(StreamOutput out, FieldDoc fieldDoc) throws IOE public static void writeScoreDoc(StreamOutput out, ScoreDoc scoreDoc) throws IOException { if (scoreDoc.getClass().equals(ScoreDoc.class) == false) { - // throw new IllegalArgumentException("This method can only be used to serialize a ScoreDoc, not a " + scoreDoc.getClass()); + throw new IllegalArgumentException("This method can only be used to serialize a ScoreDoc, not a " + scoreDoc.getClass()); } out.writeVInt(scoreDoc.doc); out.writeFloat(scoreDoc.score); From f4f8e5c7c1c23d344eb879e47de1b966008383cf Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 4 Dec 2024 20:22:01 +0100 Subject: [PATCH 024/132] nicer --- .../search/simple/SimpleSearchIT.java | 18 +++-- .../action/search/FetchSearchPhase.java | 4 +- .../search/QueryPhaseResultConsumer.java | 6 +- .../action/search/SearchActionListener.java | 2 +- .../action/search/SearchPhaseController.java | 16 ++-- .../SearchQueryThenFetchAsyncAction.java | 78 +++++++++++++++---- .../action/search/SearchRequest.java | 2 +- .../action/search/SearchShardIterator.java | 6 +- .../elasticsearch/common/lucene/Lucene.java | 15 ++++ .../search/rescore/RescorePhase.java | 1 - 10 files changed, 112 insertions(+), 36 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java index e87c4790aa665..332d522fd93ec 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java @@ -227,14 +227,16 @@ public void testRangeQueryKeyword() throws Exception { ensureGreen(); refresh(); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lte("B")), 2L); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("A").lte("B")), 1L); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lt("B")), 1L); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte(null).lt("C")), 3L); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("B").lt(null)), 2L); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt(null).lt(null)), 4L); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("").lt(null)), 4L); - assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("").lt(null)), 3L); + var coord = internalCluster().startCoordinatingOnlyNode(Settings.EMPTY); + var client = client(coord); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lte("B")), 2L); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("A").lte("B")), 1L); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lt("B")), 1L); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte(null).lt("C")), 3L); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("B").lt(null)), 2L); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt(null).lt(null)), 4L); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("").lt(null)), 4L); + assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("").lt(null)), 3L); } public void testSimpleTerminateAfterCount() throws Exception { diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index b43486c7d27fa..da98efe98a8e3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -224,8 +224,8 @@ public void innerOnResponse(FetchSearchResult result) { public void onFailure(Exception e) { try { logger.debug(() -> "[" + contextId + "] Failed to execute fetch phase", e); - progressListener.notifyFetchFailure(shardIndex, shardTarget, e); - counter.onFailure(shardIndex, shardTarget, e); + progressListener.notifyFetchFailure(shardIndex, searchShardTarget, e); + counter.onFailure(shardIndex, searchShardTarget, e); } finally { // the search context might not be cleared on the node where the fetch was executed for example // because the action was rejected by the thread pool. in this case we need to send a dedicated diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index eeaaf8e79785c..c82c810a76561 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -199,7 +199,7 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { if (aggsList != null) { aggsList.add(DelayableWriteable.referencing(batchedResult.v2().reducedAggs)); } - topDocsStats.add(batchedResult.v1(), batchedResult.v1().timedOut, batchedResult.v1().terminatedEarly); + topDocsStats.add(batchedResult.v1()); } for (QuerySearchResult result : buffer) { topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly()); @@ -562,7 +562,7 @@ public record MergeResult( static MergeResult readFrom(StreamInput in) throws IOException { return new MergeResult( in.readCollectionAsImmutableList(i -> new SearchShard(i.readOptionalString(), new ShardId(i))), - new TopDocs(Lucene.readTotalHits(in), in.readArray(Lucene::readScoreDoc, ScoreDoc[]::new)), + new TopDocs(Lucene.readTotalHits(in), in.readArray(Lucene::readScoreDocWithShardIndex, ScoreDoc[]::new)), in.readOptionalWriteable(InternalAggregations::readFrom), in.readVLong() ); @@ -575,7 +575,7 @@ public void writeTo(StreamOutput out) throws IOException { s.shardId().writeTo(o); }); Lucene.writeTotalHits(out, reducedTopDocs.totalHits); - out.writeArray(Lucene::writeScoreDoc, reducedTopDocs.scoreDocs); + out.writeArray(Lucene::writeScoreDocWithShardIndex, reducedTopDocs.scoreDocs); out.writeOptionalWriteable(reducedAggs); out.writeVLong(estimatedSize); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchActionListener.java b/server/src/main/java/org/elasticsearch/action/search/SearchActionListener.java index 237449881fba1..b44a4b7cc37b3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchActionListener.java @@ -19,7 +19,7 @@ abstract class SearchActionListener implements ActionListener { final int requestIndex; - private final SearchShardTarget searchShardTarget; + protected final SearchShardTarget searchShardTarget; protected SearchActionListener(SearchShardTarget searchShardTarget, int shardIndex) { assert shardIndex >= 0 : "shard index must be positive"; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index e0c2d90abb351..43cc3b3c84f52 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -648,7 +648,13 @@ static ReducedQueryPhase reducedQueryPhase( : new SearchProfileResultsBuilder(profileShardResults); final SortedTopDocs sortedTopDocs; if (queryPhaseRankCoordinatorContext == null) { - sortedTopDocs = sortDocs(isScrollRequest, bufferedTopDocs, from, size, reducedCompletionSuggestions); + sortedTopDocs = sortDocs( + isScrollRequest, + bufferedTopDocs, + from, + queryResults.isEmpty() ? bufferedTopDocs.size() : size, + reducedCompletionSuggestions + ); } else { ScoreDoc[] rankedDocs = queryPhaseRankCoordinatorContext.rankQueryPhaseResults( queryResults.stream().map(SearchPhaseResult::queryResult).toList(), @@ -877,7 +883,7 @@ TotalHits getTotalHits() { } } - void add(TopDocsStats other, boolean timedOut, Boolean terminatedEarly) { + void add(TopDocsStats other) { if (trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED) { totalHits += other.totalHits; if (other.totalHitsRelation == Relation.GREATER_THAN_OR_EQUAL_TO) { @@ -888,12 +894,12 @@ void add(TopDocsStats other, boolean timedOut, Boolean terminatedEarly) { if (Float.isNaN(other.maxScore) == false) { maxScore = Math.max(maxScore, other.maxScore); } - if (timedOut) { + if (other.timedOut) { this.timedOut = true; } - if (terminatedEarly != null) { + if (other.terminatedEarly != null) { if (this.terminatedEarly == null) { - this.terminatedEarly = terminatedEarly; + this.terminatedEarly = other.terminatedEarly; } else if (terminatedEarly) { this.terminatedEarly = true; } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 89089a504289e..76c86eaf77117 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -43,7 +43,6 @@ import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.internal.AliasFilter; @@ -332,14 +331,37 @@ private static ShardSearchRequest buildShardSearchRequest( return shardRequest; } + private static final class ThinSearchPhaseResult extends SearchPhaseResult { + + ThinSearchPhaseResult(ShardSearchContextId contextId, int shardIndex) { + this.contextId = contextId; + this.setShardIndex(shardIndex); + } + + ThinSearchPhaseResult(StreamInput in) throws IOException { + super(in); + contextId = new ShardSearchContextId(in); + this.setShardIndex(in.readVInt()); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + contextId.writeTo(out); + out.writeVInt(getShardIndex()); + } + } + public static class NodeQueryResponse extends TransportResponse { private final Map failedShards; + private final List results; private final QueryPhaseResultConsumer.MergeResult mergeResult; private final SearchPhaseController.TopDocsStats topDocsStats; NodeQueryResponse(StreamInput in) throws IOException { super(in); this.failedShards = in.readMap(StreamInput::readVInt, StreamInput::readException); + this.results = in.readCollectionAsImmutableList(ThinSearchPhaseResult::new); this.mergeResult = QueryPhaseResultConsumer.MergeResult.readFrom(in); this.topDocsStats = SearchPhaseController.TopDocsStats.readFrom(in); } @@ -347,16 +369,19 @@ public static class NodeQueryResponse extends TransportResponse { NodeQueryResponse( Map failedShards, QueryPhaseResultConsumer.MergeResult mergeResult, - SearchPhaseController.TopDocsStats topDocsStats + SearchPhaseController.TopDocsStats topDocsStats, + List results ) { this.failedShards = failedShards; this.mergeResult = mergeResult; this.topDocsStats = topDocsStats; + this.results = results; } @Override public void writeTo(StreamOutput out) throws IOException { out.writeMap(failedShards, StreamOutput::writeVInt, StreamOutput::writeException); + out.writeCollection(results); mergeResult.writeTo(out); topDocsStats.writeTo(out); } @@ -561,6 +586,33 @@ public void handleResponse(NodeQueryResponse response) { } if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { queryPhaseResultConsumer.reduce(response.topDocsStats, response.mergeResult); + for (ThinSearchPhaseResult result : response.results) { + var shardIt = shardsIts.get(result.getShardIndex()); + queryPhaseResultConsumer.results.set( + result.getShardIndex(), + new QuerySearchResult( + result.getContextId(), + new SearchShardTarget(nodeId, shardIt.shardId(), null), + buildShardSearchRequest( + shardIt.shardId(), + shardIt.getClusterAlias(), + result.getShardIndex(), + shardIt.getSearchContextId(), + shardIt.getOriginalIndices(), + aliasFilter.get(shardIt.shardId().getIndex().getUUID()), + shardIt.getSearchContextKeepAlive(), + concreteIndexBoosts.getOrDefault( + shardIt.shardId().getIndex().getUUID(), + DEFAULT_INDEX_BOOST + ), + request.searchRequest, + results.getNumShards(), + timeProvider.absoluteStartMillis(), + true + ) + ) + ); + } } else { if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; @@ -907,12 +959,7 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C } public boolean isPartOfPointInTime(ShardSearchContextId contextId) { - final PointInTimeBuilder pointInTimeBuilder = request.pointInTimeBuilder(); - if (pointInTimeBuilder != null) { - return request.pointInTimeBuilder().getSearchContextId(namedWriteableRegistry).contains(contextId); - } else { - return false; - } + return AbstractSearchAsyncAction.isPartOfPIT(namedWriteableRegistry, request, contextId); } private ShardSearchFailure[] buildShardFailures() { @@ -961,6 +1008,7 @@ public static void registerNodeSearchAction(TransportService transportService, S var executor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH); final ConcurrentHashMap failures = new ConcurrentHashMap<>(); final AtomicInteger shardIndex = new AtomicInteger(); + request.searchRequest.finalReduce = false; final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( request.searchRequest, executor, @@ -987,7 +1035,11 @@ public static void registerNodeSearchAction(TransportService transportService, S mergeResult.aggregations(), 0L ), - queryPhaseResultConsumer.topDocsStats + queryPhaseResultConsumer.topDocsStats, + queryPhaseResultConsumer.results.asList() + .stream() + .map(s -> new ThinSearchPhaseResult(s.getContextId(), s.getShardIndex())) + .toList() ) ); } catch (Exception e) { @@ -1003,7 +1055,7 @@ public static void registerNodeSearchAction(TransportService transportService, S executor.execute( () -> executeOne( searchService, - request, + request.searchRequest, (CancellableTask) task, shardToQuery, shardIndex, @@ -1023,7 +1075,7 @@ public static void registerNodeSearchAction(TransportService transportService, S private static void executeOne( SearchService searchService, - NodeQueryRequest request, + SearchRequest request, CancellableTask task, ShardToQuery shardToQuery, AtomicInteger shardIndex, @@ -1042,7 +1094,7 @@ private static void executeOne( AliasFilter.EMPTY, shardToQuery.contextId == null ? null : TimeValue.MAX_VALUE, shardToQuery.boost, - request.searchRequest, + request, 2, System.currentTimeMillis(), false @@ -1057,7 +1109,7 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { @Override public void onFailure(Exception e) { - failures.put(req.shardRequestIndex(), e); + failures.put(shardToQuery.shardIndex, e); queryPhaseResultConsumer.consumeShardFailure(req.shardRequestIndex()); onDone.run(); maybeNext(); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index 2e1d58e042f09..1356aa91b4a1a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -65,7 +65,7 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla private final String localClusterAlias; private final long absoluteStartMillis; - private final boolean finalReduce; + public boolean finalReduce; private SearchType searchType = SearchType.DEFAULT; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java b/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java index b543ff922886f..bc5f56f572e9f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java @@ -103,12 +103,14 @@ public String getClusterAlias() { return clusterAlias; } + public SearchShardTarget current; + SearchShardTarget nextOrNull() { final String nodeId = targetNodesIterator.nextOrNull(); if (nodeId != null) { - return new SearchShardTarget(nodeId, shardId, clusterAlias); + return current = new SearchShardTarget(nodeId, shardId, clusterAlias); } - return null; + return current = null; } int remaining() { diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index a57b8b4d23cdb..e6e6fe8cf8adb 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -389,6 +389,12 @@ public static ScoreDoc readScoreDoc(StreamInput in) throws IOException { return new ScoreDoc(in.readVInt(), in.readFloat()); } + public static ScoreDoc readScoreDocWithShardIndex(StreamInput in) throws IOException { + var res = new ScoreDoc(in.readVInt(), in.readFloat()); + res.shardIndex = in.readVInt(); + return res; + } + private static final Class GEO_DISTANCE_SORT_TYPE_CLASS = LatLonDocValuesField.newDistanceSort("some_geo_field", 0, 0).getClass(); public static void writeTotalHits(StreamOutput out, TotalHits totalHits) throws IOException { @@ -505,6 +511,15 @@ public static void writeScoreDoc(StreamOutput out, ScoreDoc scoreDoc) throws IOE out.writeFloat(scoreDoc.score); } + public static void writeScoreDocWithShardIndex(StreamOutput out, ScoreDoc scoreDoc) throws IOException { + if (scoreDoc.getClass().equals(ScoreDoc.class) == false) { + throw new IllegalArgumentException("This method can only be used to serialize a ScoreDoc, not a " + scoreDoc.getClass()); + } + out.writeVInt(scoreDoc.doc); + out.writeFloat(scoreDoc.score); + out.writeVInt(scoreDoc.shardIndex); + } + // LUCENE 4 UPGRADE: We might want to maintain our own ordinal, instead of Lucene's ordinal public static SortField.Type readSortType(StreamInput in) throws IOException { return SortField.Type.values()[in.readVInt()]; diff --git a/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java b/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java index a697960993994..bb74c857285d1 100644 --- a/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java +++ b/server/src/main/java/org/elasticsearch/search/rescore/RescorePhase.java @@ -14,7 +14,6 @@ import org.apache.lucene.search.SortField; import org.apache.lucene.search.TopDocs; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.common.lucene.search.TopDocsAndMaxScore; import org.elasticsearch.common.util.Maps; import org.elasticsearch.lucene.grouping.TopFieldGroups; From da4f888f51a6e76550e4f376dacf07d35a478637 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 5 Dec 2024 23:00:33 +0100 Subject: [PATCH 025/132] bck --- .../search/AbstractSearchAsyncAction.java | 14 +- .../action/search/FetchSearchPhase.java | 2 +- .../search/QueryPhaseResultConsumer.java | 8 +- .../SearchQueryThenFetchAsyncAction.java | 151 +++++++----------- .../search/query/QuerySearchResult.java | 11 +- 5 files changed, 81 insertions(+), 105 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index ece1c4e358451..e03d0377531fa 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -354,7 +354,7 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. */ - ShardOperationFailedException[] shardSearchFailures = buildShardFailures(); + ShardOperationFailedException[] shardSearchFailures = buildShardFailures(shardFailures); if (shardSearchFailures.length == getNumShards()) { shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = shardSearchFailures.length == 0 @@ -421,8 +421,8 @@ private void executePhase(SearchPhase phase) { } } - private ShardSearchFailure[] buildShardFailures() { - AtomicArray shardFailures = this.shardFailures.get(); + static ShardSearchFailure[] buildShardFailures(SetOnce> shardFailuresRef) { + AtomicArray shardFailures = shardFailuresRef.get(); if (shardFailures == null) { return ShardSearchFailure.EMPTY_ARRAY; } @@ -459,7 +459,7 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } else if (totalOps > expectedTotalOps) { throw new AssertionError( "unexpected higher total ops [" + totalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) + new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures(shardFailures)) ); } else { if (lastShard == false) { @@ -578,7 +578,7 @@ private void successfulShardExecution(SearchShardIterator shardsIt) { } else if (xTotalOps > expectedTotalOps) { throw new AssertionError( "unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) + new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures(shardFailures)) ); } } @@ -673,7 +673,7 @@ boolean buildPointInTimeFromSearchResults() { * @param queryResults the results of the query phase */ public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { - ShardSearchFailure[] failures = buildShardFailures(); + ShardSearchFailure[] failures = buildShardFailures(shardFailures); Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; if (allowPartialResults == false && failures.length > 0) { @@ -704,7 +704,7 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At * @param cause the cause of the phase failure */ public void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { - raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); + raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures(shardFailures))); } /** diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index da98efe98a8e3..146d455258763 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -110,7 +110,7 @@ private void innerRun() throws Exception { && context.getRequest().hasKnnSearch() == false && reducedQueryPhase.queryPhaseRankCoordinatorContext() == null && (context.getRequest().source() == null || context.getRequest().source().rankBuilder() == null); - if (queryAndFetchOptimization) { + if (false && queryAndFetchOptimization) { assert assertConsistentWithQueryAndFetchOptimization(); // query AND fetch optimization moveToNextPhase(searchPhaseShardResults, reducedQueryPhase); diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index c82c810a76561..cdb9f2fe41ee6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -161,9 +161,15 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { private final List> batchedResults = new ArrayList<>(); - public void reduce(TopDocsStats topDocsStats, MergeResult mergeResult) { + public void reduce(Object[] results, TopDocsStats topDocsStats, MergeResult mergeResult) { synchronized (this) { batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); + for (Object result : results) { + if (result instanceof QuerySearchResult querySearchResult) { + this.results.set(querySearchResult.getShardIndex(), querySearchResult); + querySearchResult.incRef(); + } + } } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 76c86eaf77117..504613f0c7c5e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -238,7 +238,7 @@ public SearchRequest getRequest() { * @param queryResults the results of the query phase */ public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { - ShardSearchFailure[] failures = buildShardFailures(); + ShardSearchFailure[] failures = AbstractSearchAsyncAction.buildShardFailures(shardFailures); Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; if (allowPartialResults == false && failures.length > 0) { @@ -331,57 +331,40 @@ private static ShardSearchRequest buildShardSearchRequest( return shardRequest; } - private static final class ThinSearchPhaseResult extends SearchPhaseResult { - - ThinSearchPhaseResult(ShardSearchContextId contextId, int shardIndex) { - this.contextId = contextId; - this.setShardIndex(shardIndex); - } - - ThinSearchPhaseResult(StreamInput in) throws IOException { - super(in); - contextId = new ShardSearchContextId(in); - this.setShardIndex(in.readVInt()); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - super.writeTo(out); - contextId.writeTo(out); - out.writeVInt(getShardIndex()); - } - } - public static class NodeQueryResponse extends TransportResponse { - private final Map failedShards; - private final List results; - private final QueryPhaseResultConsumer.MergeResult mergeResult; + private final Object[] results; private final SearchPhaseController.TopDocsStats topDocsStats; + private final QueryPhaseResultConsumer.MergeResult mergeResult; NodeQueryResponse(StreamInput in) throws IOException { super(in); - this.failedShards = in.readMap(StreamInput::readVInt, StreamInput::readException); - this.results = in.readCollectionAsImmutableList(ThinSearchPhaseResult::new); + this.results = in.readArray(i -> i.readBoolean() ? new QuerySearchResult(i) : i.readException(), Object[]::new); this.mergeResult = QueryPhaseResultConsumer.MergeResult.readFrom(in); this.topDocsStats = SearchPhaseController.TopDocsStats.readFrom(in); } NodeQueryResponse( - Map failedShards, QueryPhaseResultConsumer.MergeResult mergeResult, - SearchPhaseController.TopDocsStats topDocsStats, - List results + Object[] results, + SearchPhaseController.TopDocsStats topDocsStats ) { - this.failedShards = failedShards; + this.results = results; this.mergeResult = mergeResult; this.topDocsStats = topDocsStats; - this.results = results; } @Override public void writeTo(StreamOutput out) throws IOException { - out.writeMap(failedShards, StreamOutput::writeVInt, StreamOutput::writeException); - out.writeCollection(results); + out.writeArray((o, v) -> { + if (v instanceof Exception e) { + o.writeBoolean(false); + o.writeException(e); + } else { + o.writeBoolean(true); + assert v instanceof QuerySearchResult : v; + ((QuerySearchResult) v).writeTo(o); + } + }, results); mergeResult.writeTo(out); topDocsStats.writeTo(out); } @@ -573,46 +556,22 @@ public Executor executor() { @Override public void handleResponse(NodeQueryResponse response) { - response.failedShards.forEach( - (sIdx, e) -> onShardFailure( - sIdx, - new SearchShardTarget(nodeId, shardIterators[sIdx].shardId(), null), - e - ) - ); - final int successfulShards = request.shards.size() - response.failedShards.size(); + int failedShards = 0; + for (int i = 0; i < response.results.length; i++) { + var s = request.shards.get(i); + if (response.results[i] instanceof Exception e) { + onShardFailure(s.shardIndex, new SearchShardTarget(nodeId, s.shardId(), null), e); + failedShards++; + } else if (response.results[i] instanceof QuerySearchResult q) { + q.setShardIndex(s.shardIndex); + } + } + final int successfulShards = request.shards.size() - failedShards; if (successfulShards > 0) { hasShardResponse.set(true); } if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { - queryPhaseResultConsumer.reduce(response.topDocsStats, response.mergeResult); - for (ThinSearchPhaseResult result : response.results) { - var shardIt = shardsIts.get(result.getShardIndex()); - queryPhaseResultConsumer.results.set( - result.getShardIndex(), - new QuerySearchResult( - result.getContextId(), - new SearchShardTarget(nodeId, shardIt.shardId(), null), - buildShardSearchRequest( - shardIt.shardId(), - shardIt.getClusterAlias(), - result.getShardIndex(), - shardIt.getSearchContextId(), - shardIt.getOriginalIndices(), - aliasFilter.get(shardIt.shardId().getIndex().getUUID()), - shardIt.getSearchContextKeepAlive(), - concreteIndexBoosts.getOrDefault( - shardIt.shardId().getIndex().getUUID(), - DEFAULT_INDEX_BOOST - ), - request.searchRequest, - results.getNumShards(), - timeProvider.absoluteStartMillis(), - true - ) - ) - ); - } + queryPhaseResultConsumer.reduce(response.results, response.topDocsStats, response.mergeResult); } else { if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; @@ -626,7 +585,7 @@ public void handleResponse(NodeQueryResponse response) { } final int numShards = results.getNumShards(); successfulOps.addAndGet(successfulShards); - final int total = totalOps.addAndGet(successfulShards + response.failedShards.size()); + final int total = totalOps.addAndGet(successfulShards + failedShards); if (total == numShards) { onPhaseDone(); } else if (total > numShards) { @@ -745,7 +704,12 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } else if (totalOps > expectedTotalOps) { throw new AssertionError( "unexpected higher total ops [" + totalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) + new SearchPhaseExecutionException( + getName(), + "Shard failures", + null, + AbstractSearchAsyncAction.buildShardFailures(shardFailures) + ) ); } else { if (lastShard == false) { @@ -825,7 +789,12 @@ private void successfulShardExecution(SearchShardIterator shardsIt) { } else if (xTotalOps > expectedTotalOps) { throw new AssertionError( "unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) + new SearchPhaseExecutionException( + getName(), + "Shard failures", + null, + AbstractSearchAsyncAction.buildShardFailures(shardFailures) + ) ); } } @@ -840,7 +809,7 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. */ - ShardOperationFailedException[] shardSearchFailures = buildShardFailures(); + ShardOperationFailedException[] shardSearchFailures = AbstractSearchAsyncAction.buildShardFailures(shardFailures); if (shardSearchFailures.length == results.getNumShards()) { shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = shardSearchFailures.length == 0 @@ -915,7 +884,9 @@ private void executePhase(SearchPhase phase) { * @param cause the cause of the phase failure */ public void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { - raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures())); + raisePhaseFailure( + new SearchPhaseExecutionException(phase.getName(), msg, cause, AbstractSearchAsyncAction.buildShardFailures(shardFailures)) + ); } @Override @@ -962,19 +933,6 @@ public boolean isPartOfPointInTime(ShardSearchContextId contextId) { return AbstractSearchAsyncAction.isPartOfPIT(namedWriteableRegistry, request, contextId); } - private ShardSearchFailure[] buildShardFailures() { - AtomicArray shardFailures = this.shardFailures.get(); - if (shardFailures == null) { - return ShardSearchFailure.EMPTY_ARRAY; - } - List entries = shardFailures.asList(); - ShardSearchFailure[] failures = new ShardSearchFailure[entries.size()]; - for (int i = 0; i < failures.length; i++) { - failures[i] = entries.get(i); - } - return failures; - } - private void onShardResultConsumed(SearchPhaseResult result, SearchShardIterator shardIt) { successfulOps.incrementAndGet(); // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level @@ -1026,20 +984,25 @@ public static void registerNodeSearchAction(TransportService transportService, S if (countDown.countDown()) { try { var mergeResult = queryPhaseResultConsumer.reduce(); + final Object[] results = new Object[request.shards.size()]; + for (int i = 0; i < results.length; i++) { + var e = failures.get(i); + if (e != null) { + results[i] = e; + } else { + results[i] = queryPhaseResultConsumer.results.get(i); + } + } new ChannelActionListener<>(channel).onResponse( new NodeQueryResponse( - Map.copyOf(failures), new QueryPhaseResultConsumer.MergeResult( request.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), new TopDocs(mergeResult.totalHits(), mergeResult.sortedTopDocs().scoreDocs()), mergeResult.aggregations(), 0L ), - queryPhaseResultConsumer.topDocsStats, - queryPhaseResultConsumer.results.asList() - .stream() - .map(s -> new ThinSearchPhaseResult(s.getContextId(), s.getShardIndex())) - .toList() + results, + queryPhaseResultConsumer.topDocsStats ) ); } catch (Exception e) { diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index b7e2e361c28b6..c877035d83e67 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -381,7 +381,9 @@ private void readFromWithId(ShardSearchContextId id, StreamInput in, boolean del sortValueFormats[i] = in.readNamedWriteable(DocValueFormat.class); } } - setTopDocs(readTopDocs(in)); + if (in.readBoolean()) { + setTopDocs(readTopDocs(in)); + } hasAggs = in.readBoolean(); boolean success = false; try { @@ -439,7 +441,12 @@ public void writeToNoId(StreamOutput out) throws IOException { out.writeNamedWriteable(sortValueFormats[i]); } } - writeTopDocs(out, topDocsAndMaxScore); + if (topDocsAndMaxScore == null) { + out.writeBoolean(false); + } else { + out.writeBoolean(true); + writeTopDocs(out, topDocsAndMaxScore); + } out.writeOptionalWriteable(aggregations); if (suggest == null) { out.writeBoolean(false); From 72273094f8fd6c5b85f83a44068c19a22588c91d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 6 Dec 2024 03:38:41 +0100 Subject: [PATCH 026/132] meh --- .../action/search/SearchQueryThenFetchAsyncAction.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 504613f0c7c5e..b0d9ae3800e90 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -564,6 +564,7 @@ public void handleResponse(NodeQueryResponse response) { failedShards++; } else if (response.results[i] instanceof QuerySearchResult q) { q.setShardIndex(s.shardIndex); + q.setSearchShardTarget(new SearchShardTarget(nodeId, s.shardId(), null)); } } final int successfulShards = request.shards.size() - failedShards; From 2935eb10f57a0e129be02479194bd048aed5eb71 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 9 Dec 2024 14:15:40 +0100 Subject: [PATCH 027/132] bring back optimization --- .../action/search/FetchSearchPhase.java | 8 ++++---- .../search/SearchQueryThenFetchAsyncAction.java | 12 +++++++++++- 2 files changed, 15 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 146d455258763..2de1e5f4e8b06 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -104,13 +104,13 @@ private void innerRun() throws Exception { assert this.reducedQueryPhase == null ^ this.resultConsumer == null; // depending on whether we executed the RankFeaturePhase we may or may not have the reduced query result computed already final var reducedQueryPhase = this.reducedQueryPhase == null ? resultConsumer.reduce() : this.reducedQueryPhase; + var request = context.getRequest(); // Usually when there is a single shard, we force the search type QUERY_THEN_FETCH. But when there's kNN, we might // still use DFS_QUERY_THEN_FETCH, which does not perform the "query and fetch" optimization during the query phase. - final boolean queryAndFetchOptimization = numShards == 1 - && context.getRequest().hasKnnSearch() == false + if (numShards == 1 + && request.hasKnnSearch() == false && reducedQueryPhase.queryPhaseRankCoordinatorContext() == null - && (context.getRequest().source() == null || context.getRequest().source().rankBuilder() == null); - if (false && queryAndFetchOptimization) { + && (request.source() == null || request.source().rankBuilder() == null)) { assert assertConsistentWithQueryAndFetchOptimization(); // query AND fetch optimization moveToNextPhase(searchPhaseShardResults, reducedQueryPhase); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index b0d9ae3800e90..0d42f4d18145f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -517,7 +517,7 @@ public void run() throws IOException { if (routing == null) { failOnUnavailable(shardIndex, shardRoutings); } else { - if (minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { + if (routing.getClusterAlias() == null && minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { perNodeQueries.computeIfAbsent( routing.getNodeId(), ignored -> new NodeQueryRequest(new ArrayList<>(), request) @@ -536,6 +536,16 @@ public void run() throws IOException { } } perNodeQueries.forEach((nodeId, request) -> { + if (request.shards.size() == 1) { + var shard = request.shards.getFirst(); + this.performPhaseOnShard( + shard.shardIndex, + shardIterators[shard.shardIndex], + new SearchShardTarget(nodeId, shard.shardId, null) + ); + return; + } + try { searchTransportService.transportService() .sendChildRequest( From 827fb4d1bfdd544c615500fb58d7fe7798de2191 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 9 Dec 2024 14:29:14 +0100 Subject: [PATCH 028/132] bck --- .../action/search/QueryPhaseResultConsumer.java | 5 +++-- .../action/search/SearchPhaseController.java | 8 +------- .../action/search/SearchQueryThenFetchAsyncAction.java | 10 ++++++---- 3 files changed, 10 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index cdb9f2fe41ee6..e26a155026023 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -76,7 +76,7 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults onPartialMergeFailure; private final int batchReduceSize; - private final List buffer = new ArrayList<>(); + final List buffer = new ArrayList<>(); private final List emptyResults = new ArrayList<>(); // the memory that is accounted in the circuit breaker for this consumer private volatile long circuitBreakerBytes; @@ -163,11 +163,12 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { public void reduce(Object[] results, TopDocsStats topDocsStats, MergeResult mergeResult) { synchronized (this) { - batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); + //batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); for (Object result : results) { if (result instanceof QuerySearchResult querySearchResult) { this.results.set(querySearchResult.getShardIndex(), querySearchResult); querySearchResult.incRef(); + buffer.add(querySearchResult); } } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 43cc3b3c84f52..cd7749c22222f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -648,13 +648,7 @@ static ReducedQueryPhase reducedQueryPhase( : new SearchProfileResultsBuilder(profileShardResults); final SortedTopDocs sortedTopDocs; if (queryPhaseRankCoordinatorContext == null) { - sortedTopDocs = sortDocs( - isScrollRequest, - bufferedTopDocs, - from, - queryResults.isEmpty() ? bufferedTopDocs.size() : size, - reducedCompletionSuggestions - ); + sortedTopDocs = sortDocs(isScrollRequest, bufferedTopDocs, from, size, reducedCompletionSuggestions); } else { ScoreDoc[] rankedDocs = queryPhaseRankCoordinatorContext.rankQueryPhaseResults( queryResults.stream().map(SearchPhaseResult::queryResult).toList(), diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 0d42f4d18145f..c1e6057d1481e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -10,7 +10,6 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; -import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.SetOnce; @@ -32,6 +31,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.EsExecutors; @@ -978,6 +978,7 @@ public static void registerNodeSearchAction(TransportService transportService, S final ConcurrentHashMap failures = new ConcurrentHashMap<>(); final AtomicInteger shardIndex = new AtomicInteger(); request.searchRequest.finalReduce = false; + request.searchRequest.setBatchedReduceSize(Integer.MAX_VALUE); final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( request.searchRequest, executor, @@ -994,7 +995,6 @@ public static void registerNodeSearchAction(TransportService transportService, S final Runnable onDone = () -> { if (countDown.countDown()) { try { - var mergeResult = queryPhaseResultConsumer.reduce(); final Object[] results = new Object[request.shards.size()]; for (int i = 0; i < results.length; i++) { var e = failures.get(i); @@ -1004,12 +1004,14 @@ public static void registerNodeSearchAction(TransportService transportService, S results[i] = queryPhaseResultConsumer.results.get(i); } } + // TODO: facepalm + queryPhaseResultConsumer.buffer.clear(); new ChannelActionListener<>(channel).onResponse( new NodeQueryResponse( new QueryPhaseResultConsumer.MergeResult( request.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), - new TopDocs(mergeResult.totalHits(), mergeResult.sortedTopDocs().scoreDocs()), - mergeResult.aggregations(), + Lucene.EMPTY_TOP_DOCS, + null, 0L ), results, From de414a22dba96f0a84adb438c13ad99724d0e6e6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 9 Dec 2024 15:02:19 +0100 Subject: [PATCH 029/132] bck --- .../action/search/QueryPhaseResultConsumer.java | 2 +- .../search/SearchQueryThenFetchAsyncAction.java | 16 ++++++++++++---- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index e26a155026023..35309bd36e1bd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -163,7 +163,7 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { public void reduce(Object[] results, TopDocsStats topDocsStats, MergeResult mergeResult) { synchronized (this) { - //batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); + // batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); for (Object result : results) { if (result instanceof QuerySearchResult querySearchResult) { this.results.set(querySearchResult.getShardIndex(), querySearchResult); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index c1e6057d1481e..408f6281b8a4e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -575,6 +575,7 @@ public void handleResponse(NodeQueryResponse response) { } else if (response.results[i] instanceof QuerySearchResult q) { q.setShardIndex(s.shardIndex); q.setSearchShardTarget(new SearchShardTarget(nodeId, s.shardId(), null)); + totalOps.addAndGet(shardIterators[s.shardIndex].remaining()); } } final int successfulShards = request.shards.size() - failedShards; @@ -585,21 +586,28 @@ public void handleResponse(NodeQueryResponse response) { queryPhaseResultConsumer.reduce(response.results, response.topDocsStats, response.mergeResult); } else { if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { - var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; + /* + TODO: do this + var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; countOnlyQueryPhaseResultConsumer.reduce( false, false, reducedTotalHits.value(), reducedTotalHits.relation() ); + */ + for (Object result : response.results) { + if (result instanceof SearchPhaseResult searchPhaseResult) { + countOnlyQueryPhaseResultConsumer.consumeResult(searchPhaseResult, () -> {}); + } + } } } - final int numShards = results.getNumShards(); successfulOps.addAndGet(successfulShards); final int total = totalOps.addAndGet(successfulShards + failedShards); - if (total == numShards) { + if (total == expectedTotalOps) { onPhaseDone(); - } else if (total > numShards) { + } else if (total > expectedTotalOps) { throw new AssertionError(); } } From 161f1f7e066beaf1607d2d2b22361faed8863d84 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 9 Dec 2024 22:20:59 +0100 Subject: [PATCH 030/132] fix --- .../action/search/SearchQueryThenFetchAsyncAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 408f6281b8a4e..6d65a59301908 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -1005,7 +1005,8 @@ public static void registerNodeSearchAction(TransportService transportService, S try { final Object[] results = new Object[request.shards.size()]; for (int i = 0; i < results.length; i++) { - var e = failures.get(i); + int shardIdx = request.shards.get(i).shardIndex; + var e = failures.get(shardIdx); if (e != null) { results[i] = e; } else { From 5b5ef3244e8b207256654976b4573295222cfe42 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 10 Dec 2024 00:27:59 +0100 Subject: [PATCH 031/132] works more --- .../action/search/SearchQueryThenFetchAsyncAction.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 6d65a59301908..2e74ed2a13866 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -569,11 +569,12 @@ public void handleResponse(NodeQueryResponse response) { int failedShards = 0; for (int i = 0; i < response.results.length; i++) { var s = request.shards.get(i); + int shardIdx = s.shardIndex; if (response.results[i] instanceof Exception e) { - onShardFailure(s.shardIndex, new SearchShardTarget(nodeId, s.shardId(), null), e); + onShardFailure(shardIdx, new SearchShardTarget(nodeId, s.shardId(), null), e); failedShards++; } else if (response.results[i] instanceof QuerySearchResult q) { - q.setShardIndex(s.shardIndex); + q.setShardIndex(shardIdx); q.setSearchShardTarget(new SearchShardTarget(nodeId, s.shardId(), null)); totalOps.addAndGet(shardIterators[s.shardIndex].remaining()); } @@ -1035,8 +1036,8 @@ public static void registerNodeSearchAction(TransportService transportService, S } }; for (int i = 0; i < workers; i++) { - ShardToQuery shardToQuery; - if ((shardToQuery = shards.poll()) != null) { + ShardToQuery shardToQuery = shards.poll(); + if (shardToQuery != null) { executor.execute( () -> executeOne( searchService, From 51c113fc517773d94a4b39c307404aa9f9763e5f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 10 Dec 2024 16:52:49 +0100 Subject: [PATCH 032/132] bck --- .../search/AbstractSearchAsyncAction.java | 1 - .../action/search/SearchPhaseResults.java | 2 - .../SearchQueryThenFetchAsyncAction.java | 141 ++++++++---------- 3 files changed, 62 insertions(+), 82 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index e03d0377531fa..95bb49d9a20f9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -522,7 +522,6 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter } } - results.consumeShardFailure(shardIndex); } static boolean isTaskCancelledException(Exception e) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseResults.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseResults.java index 73bb0d545a2e0..54f9f5549f30c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseResults.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseResults.java @@ -49,8 +49,6 @@ final int getNumShards() { */ abstract boolean hasResult(int shardIndex); - void consumeShardFailure(int shardIndex) {} - AtomicArray getAtomicArray() { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 2e74ed2a13866..a12a395cd5c5f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -67,6 +67,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; @@ -112,10 +113,10 @@ class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearch protected final GroupShardsIterator shardsIts; private final SearchShardIterator[] shardIterators; private final Map shardIndexMap; - private final int expectedTotalOps; - private final AtomicInteger totalOps = new AtomicInteger(); private final AtomicBoolean requestCancelled = new AtomicBoolean(); + private final Set outstandingShards = Collections.newSetFromMap(new ConcurrentHashMap<>()); + // protected for tests protected final List releasables = new ArrayList<>(); @@ -170,12 +171,6 @@ class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearch } this.shardIndexMap = Collections.unmodifiableMap(shardMap); this.shardIterators = searchIterators.toArray(SearchShardIterator[]::new); - - // we need to add 1 for non active partition, since we count it in the total. This means for each shard in the iterator we sum up - // it's number of active shards but use 1 as the default if no replica of a shard is active at this point. - // on a per shards level we use shardIt.remaining() to increment the totalOps pointer but add 1 for the current shard result - // we process hence we add one for the non active partition here. - this.expectedTotalOps = shardsIts.totalSizeWith1ForEmpty(); this.timeProvider = timeProvider; this.logger = logger; this.searchTransportService = searchTransportService; @@ -501,6 +496,13 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) @Override public void run() throws IOException { + // TODO: stupid + for (int i = 0; i < shardsIts.size(); i++) { + outstandingShards.add(shardsIts.get(i).shardId()); + } + for (SearchShardIterator toSkipShardsIt : toSkipShardsIts) { + outstandingShards.add(toSkipShardsIt.shardId()); + } for (final SearchShardIterator iterator : toSkipShardsIts) { assert iterator.skip(); skipShard(iterator); @@ -571,45 +573,55 @@ public void handleResponse(NodeQueryResponse response) { var s = request.shards.get(i); int shardIdx = s.shardIndex; if (response.results[i] instanceof Exception e) { - onShardFailure(shardIdx, new SearchShardTarget(nodeId, s.shardId(), null), e); + onShardFailure( + shardIdx, + new SearchShardTarget(nodeId, s.shardId(), null), + shardIterators[shardIdx], + e + ); failedShards++; } else if (response.results[i] instanceof QuerySearchResult q) { q.setShardIndex(shardIdx); - q.setSearchShardTarget(new SearchShardTarget(nodeId, s.shardId(), null)); - totalOps.addAndGet(shardIterators[s.shardIndex].remaining()); + var shardId = s.shardId(); + q.setSearchShardTarget(new SearchShardTarget(nodeId, shardId, null)); } } final int successfulShards = request.shards.size() - failedShards; if (successfulShards > 0) { hasShardResponse.set(true); - } - if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { - queryPhaseResultConsumer.reduce(response.results, response.topDocsStats, response.mergeResult); - } else { - if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { - /* - TODO: do this - var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; - countOnlyQueryPhaseResultConsumer.reduce( - false, - false, - reducedTotalHits.value(), - reducedTotalHits.relation() - ); - */ - for (Object result : response.results) { - if (result instanceof SearchPhaseResult searchPhaseResult) { - countOnlyQueryPhaseResultConsumer.consumeResult(searchPhaseResult, () -> {}); + if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { + queryPhaseResultConsumer.reduce(response.results, response.topDocsStats, response.mergeResult); + } else { + if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { + /* + TODO: do this + var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; + countOnlyQueryPhaseResultConsumer.reduce( + false, + false, + reducedTotalHits.value(), + reducedTotalHits.relation() + ); + */ + for (Object result : response.results) { + if (result instanceof SearchPhaseResult searchPhaseResult) { + countOnlyQueryPhaseResultConsumer.consumeResult(searchPhaseResult, () -> {}); + } } } } - } - successfulOps.addAndGet(successfulShards); - final int total = totalOps.addAndGet(successfulShards + failedShards); - if (total == expectedTotalOps) { - onPhaseDone(); - } else if (total > expectedTotalOps) { - throw new AssertionError(); + successfulOps.addAndGet(successfulShards); + for (Object result : response.results) { + if (result instanceof SearchPhaseResult searchPhaseResult) { + boolean removed = outstandingShards.remove( + searchPhaseResult.getSearchShardTarget().getShardId() + ); + assert removed; + } + } + if (outstandingShards.isEmpty()) { + onPhaseDone(); + } } } @@ -718,23 +730,19 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } onShardGroupFailure(shardIndex, shard, e); } - final int totalOps = this.totalOps.incrementAndGet(); - if (totalOps == expectedTotalOps) { - onPhaseDone(); - } else if (totalOps > expectedTotalOps) { - throw new AssertionError( - "unexpected higher total ops [" + totalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException( - getName(), - "Shard failures", - null, - AbstractSearchAsyncAction.buildShardFailures(shardFailures) - ) - ); + if (lastShard == false) { + performPhaseOnShard(shardIndex, shardIt, nextShard); } else { - if (lastShard == false) { - performPhaseOnShard(shardIndex, shardIt, nextShard); - } + final ShardId shardId = shardIt.shardId(); + finishShardAndMaybePhase(shardId); + } + } + + private void finishShardAndMaybePhase(ShardId shardId) { + boolean removed = outstandingShards.remove(shardId); + assert removed : "unknown shardId " + shardId; + if (outstandingShards.isEmpty()) { + onPhaseDone(); } } @@ -782,7 +790,6 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter } } - results.consumeShardFailure(shardIndex); } void skipShard(SearchShardIterator iterator) { @@ -793,30 +800,7 @@ void skipShard(SearchShardIterator iterator) { } private void successfulShardExecution(SearchShardIterator shardsIt) { - final int remainingOpsOnIterator; - if (shardsIt.skip()) { - // It's possible that we're skipping a shard that's unavailable - // but its range was available in the IndexMetadata, in that - // case the shardsIt.remaining() would be 0, expectedTotalOps - // accounts for unavailable shards too. - remainingOpsOnIterator = Math.max(shardsIt.remaining(), 1); - } else { - remainingOpsOnIterator = shardsIt.remaining() + 1; - } - final int xTotalOps = totalOps.addAndGet(remainingOpsOnIterator); - if (xTotalOps == expectedTotalOps) { - onPhaseDone(); - } else if (xTotalOps > expectedTotalOps) { - throw new AssertionError( - "unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException( - getName(), - "Shard failures", - null, - AbstractSearchAsyncAction.buildShardFailures(shardFailures) - ) - ); - } + finishShardAndMaybePhase(shardsIt.shardId()); } final void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() @@ -1078,7 +1062,7 @@ private static void executeOne( shardToQuery.contextId, shardToQuery.originalIndices, AliasFilter.EMPTY, - shardToQuery.contextId == null ? null : TimeValue.MAX_VALUE, + null, shardToQuery.boost, request, 2, @@ -1096,7 +1080,6 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { @Override public void onFailure(Exception e) { failures.put(shardToQuery.shardIndex, e); - queryPhaseResultConsumer.consumeShardFailure(req.shardRequestIndex()); onDone.run(); maybeNext(); } From 9bb15fc77d8a7f8658007f17b8628ca4a87b0d21 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 10 Dec 2024 17:31:38 +0100 Subject: [PATCH 033/132] fixes --- .../java/org/elasticsearch/action/IndicesRequestIT.java | 2 +- .../action/search/SearchQueryThenFetchAsyncAction.java | 8 ++------ 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java index f5860cedcd989..7fe14753b3fa3 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java @@ -621,7 +621,7 @@ private static void assertSameIndices(IndicesRequest originalRequest, boolean op assertThat(internalRequest.getClass().getName(), indicesRequest.indices(), equalTo(originalRequest.indices())); assertThat(indicesRequest.indicesOptions(), equalTo(originalRequest.indicesOptions())); } - } + }erlang otp } private static void assertIndicesSubset(List indices, String... actions) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index a12a395cd5c5f..c07456c933c23 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -62,12 +62,7 @@ import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; +import java.util.*; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; @@ -346,6 +341,7 @@ public static class NodeQueryResponse extends TransportResponse { this.results = results; this.mergeResult = mergeResult; this.topDocsStats = topDocsStats; + assert Arrays.stream(results).noneMatch(Objects::isNull) : Arrays.asList(results); } @Override From 57883af91846a064137d83384b260a9de6e2dec4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 10 Dec 2024 19:37:19 +0100 Subject: [PATCH 034/132] bck --- .../java/org/elasticsearch/action/IndicesRequestIT.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java index 7fe14753b3fa3..f5860cedcd989 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java @@ -621,7 +621,7 @@ private static void assertSameIndices(IndicesRequest originalRequest, boolean op assertThat(internalRequest.getClass().getName(), indicesRequest.indices(), equalTo(originalRequest.indices())); assertThat(indicesRequest.indicesOptions(), equalTo(originalRequest.indicesOptions())); } - }erlang otp + } } private static void assertIndicesSubset(List indices, String... actions) { From 7782e67e4ee27e294ebb037d0ed94de4f45c682c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 11 Dec 2024 01:48:32 +0100 Subject: [PATCH 035/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 124 +++++++++++------- .../action/search/SearchTransportService.java | 3 +- .../action/search/TransportSearchAction.java | 4 +- 3 files changed, 77 insertions(+), 54 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index c07456c933c23..76ae4b5b7cd9e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.search.TopFieldDocs; -import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; @@ -32,7 +31,10 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.util.Maps; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.Releasable; @@ -59,10 +61,15 @@ import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponseHandler; -import org.elasticsearch.transport.TransportService; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; @@ -107,10 +114,9 @@ class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearch protected final GroupShardsIterator toSkipShardsIts; protected final GroupShardsIterator shardsIts; private final SearchShardIterator[] shardIterators; - private final Map shardIndexMap; private final AtomicBoolean requestCancelled = new AtomicBoolean(); - private final Set outstandingShards = Collections.newSetFromMap(new ConcurrentHashMap<>()); + private final Set outstandingShards = ConcurrentCollections.newConcurrentSet(); // protected for tests protected final List releasables = new ArrayList<>(); @@ -155,17 +161,11 @@ class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearch this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); this.shardsIts = new GroupShardsIterator<>(iterators); + this.shardIterators = iterators.toArray(new SearchShardIterator[0]); // we compute the shard index based on the natural order of the shards // that participate in the search request. This means that this number is // consistent between two requests that target the same shards. - Map shardMap = new HashMap<>(); - List searchIterators = new ArrayList<>(iterators); - CollectionUtil.timSort(searchIterators); - for (int i = 0; i < searchIterators.size(); i++) { - shardMap.put(searchIterators.get(i), i); - } - this.shardIndexMap = Collections.unmodifiableMap(shardMap); - this.shardIterators = searchIterators.toArray(SearchShardIterator[]::new); + Arrays.sort(shardIterators); this.timeProvider = timeProvider; this.logger = logger; this.searchTransportService = searchTransportService; @@ -492,13 +492,16 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) @Override public void run() throws IOException { - // TODO: stupid - for (int i = 0; i < shardsIts.size(); i++) { - outstandingShards.add(shardsIts.get(i).shardId()); - } + // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging for (SearchShardIterator toSkipShardsIt : toSkipShardsIts) { outstandingShards.add(toSkipShardsIt.shardId()); } + final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); + for (int i = 0; i < shardIterators.length; i++) { + var iterator = shardIterators[i]; + shardIndexMap.put(iterator, i); + outstandingShards.add(iterator.shardId()); + } for (final SearchShardIterator iterator : toSkipShardsIts) { assert iterator.skip(); skipShard(iterator); @@ -585,39 +588,15 @@ public void handleResponse(NodeQueryResponse response) { final int successfulShards = request.shards.size() - failedShards; if (successfulShards > 0) { hasShardResponse.set(true); - if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { - queryPhaseResultConsumer.reduce(response.results, response.topDocsStats, response.mergeResult); - } else { - if (results instanceof CountOnlyQueryPhaseResultConsumer countOnlyQueryPhaseResultConsumer) { - /* - TODO: do this - var reducedTotalHits = response.mergeResult.reducedTopDocs().totalHits; - countOnlyQueryPhaseResultConsumer.reduce( - false, - false, - reducedTotalHits.value(), - reducedTotalHits.relation() - ); - */ - for (Object result : response.results) { - if (result instanceof SearchPhaseResult searchPhaseResult) { - countOnlyQueryPhaseResultConsumer.consumeResult(searchPhaseResult, () -> {}); - } - } - } - } - successfulOps.addAndGet(successfulShards); for (Object result : response.results) { if (result instanceof SearchPhaseResult searchPhaseResult) { - boolean removed = outstandingShards.remove( - searchPhaseResult.getSearchShardTarget().getShardId() + results.consumeResult( + searchPhaseResult, + () -> finishShardAndMaybePhase(searchPhaseResult.getSearchShardTarget().getShardId()) ); - assert removed; } } - if (outstandingShards.isEmpty()) { - onPhaseDone(); - } + successfulOps.addAndGet(successfulShards); } } @@ -952,7 +931,8 @@ private void onShardResultConsumed(SearchPhaseResult result, SearchShardIterator public static final String NODE_SEARCH_ACTION_NAME = "indices:data/read/search[query][n]"; - public static void registerNodeSearchAction(TransportService transportService, SearchService searchService) { + public static void registerNodeSearchAction(SearchTransportService searchTransportService, SearchService searchService) { + var transportService = searchTransportService.transportService(); transportService.registerRequestHandler( NODE_SEARCH_ACTION_NAME, EsExecutors.DIRECT_EXECUTOR_SERVICE, @@ -976,9 +956,7 @@ public static void registerNodeSearchAction(TransportService transportService, S ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, request.shards.size(), - e -> { - throw new AssertionError(e); - } + e -> searchTransportService.cancelSearchTask((SearchTask) task, "failed to merge result [" + e.getMessage() + "]") ); final CountDown countDown = new CountDown(request.shards.size()); final Runnable onDone = () -> { @@ -1019,7 +997,7 @@ public static void registerNodeSearchAction(TransportService transportService, S ShardToQuery shardToQuery = shards.poll(); if (shardToQuery != null) { executor.execute( - () -> executeOne( + shardTask( searchService, request.searchRequest, (CancellableTask) task, @@ -1084,7 +1062,7 @@ private void maybeNext() { var shardToQuery = shards.poll(); if (shardToQuery != null) { executor.execute( - () -> executeOne( + shardTask( searchService, request, task, @@ -1101,4 +1079,48 @@ private void maybeNext() { } }); } + + private static AbstractRunnable shardTask( + SearchService searchService, + SearchRequest request, + CancellableTask task, + ShardToQuery shardToQuery, + AtomicInteger shardIndex, + BlockingQueue shards, + Executor executor, + QueryPhaseResultConsumer queryPhaseResultConsumer, + Map failures, + Runnable onDone + ) { + return new AbstractRunnable() { + @Override + protected void doRun() { + executeOne( + searchService, + request, + task, + shardToQuery, + shardIndex, + shards, + executor, + queryPhaseResultConsumer, + failures, + onDone + ); + } + + @Override + public void onFailure(Exception e) { + throw new AssertionError("shouldn't throw", e); + } + + @Override + public void onRejection(Exception e) { + // TODO this could be done better now, we probably should only make sure to have a single loop running at + // minimum and ignore + requeue rejections in that case + failures.put(shardToQuery.shardIndex, e); + onDone.run(); + } + }; + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 379e46a8d09fe..3a553b84f2f9f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -383,12 +383,13 @@ public void writeTo(StreamOutput out) throws IOException { } } - public static void registerRequestHandler(TransportService transportService, SearchService searchService) { + public static void registerRequestHandler(SearchTransportService searchTransportService, SearchService searchService) { final TransportRequestHandler freeContextHandler = (request, channel, task) -> { logger.trace("releasing search context [{}]", request.id()); boolean freed = searchService.freeReaderContext(request.id()); channel.sendResponse(SearchFreeContextResponse.of(freed)); }; + var transportService = searchTransportService.transportService; final Executor freeContextExecutor = buildFreeContextExecutor(transportService); transportService.registerRequestHandler( FREE_CONTEXT_SCROLL_ACTION_NAME, diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index cc550080b7535..b166b49194efa 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -190,8 +190,8 @@ public TransportSearchAction( this.searchPhaseController = searchPhaseController; this.searchTransportService = searchTransportService; this.remoteClusterService = searchTransportService.getRemoteClusterService(); - SearchTransportService.registerRequestHandler(transportService, searchService); - SearchQueryThenFetchAsyncAction.registerNodeSearchAction(transportService, searchService); + SearchTransportService.registerRequestHandler(searchTransportService, searchService); + SearchQueryThenFetchAsyncAction.registerNodeSearchAction(searchTransportService, searchService); this.clusterService = clusterService; this.transportService = transportService; this.searchService = searchService; From 9f7bb0784b4c8dda0f576b5760e6b0efbec89e65 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 11 Dec 2024 02:13:50 +0100 Subject: [PATCH 036/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 41 +++++++++++++++---- 1 file changed, 34 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 76ae4b5b7cd9e..38c9903123e62 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -364,16 +364,19 @@ public void writeTo(StreamOutput out) throws IOException { public static class NodeQueryRequest extends TransportRequest { private final List shards; private final SearchRequest searchRequest; + private final Map aliasFilters; - private NodeQueryRequest(List shards, SearchRequest searchRequest) { + private NodeQueryRequest(List shards, SearchRequest searchRequest, Map aliasFilters) { this.shards = shards; this.searchRequest = searchRequest; + this.aliasFilters = aliasFilters; } private NodeQueryRequest(StreamInput in) throws IOException { super(in); this.shards = in.readCollectionAsImmutableList(ShardToQuery::readFrom); this.searchRequest = new SearchRequest(in); + this.aliasFilters = in.readImmutableMap(AliasFilter::readFrom); } @Override @@ -386,6 +389,7 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeCollection(shards); searchRequest.writeTo(out); + out.writeMap(aliasFilters, (o, v) -> v.writeTo(o)); } } @@ -521,7 +525,7 @@ public void run() throws IOException { if (routing.getClusterAlias() == null && minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { perNodeQueries.computeIfAbsent( routing.getNodeId(), - ignored -> new NodeQueryRequest(new ArrayList<>(), request) + ignored -> new NodeQueryRequest(new ArrayList<>(), request, aliasFilter) ).shards.add( new ShardToQuery( concreteIndexBoosts.getOrDefault(routing.getShardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), @@ -970,6 +974,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo results[i] = e; } else { results[i] = queryPhaseResultConsumer.results.get(i); + assert results[i] != null; } } // TODO: facepalm @@ -999,7 +1004,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo executor.execute( shardTask( searchService, - request.searchRequest, + request, (CancellableTask) task, shardToQuery, shardIndex, @@ -1019,7 +1024,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo private static void executeOne( SearchService searchService, - SearchRequest request, + NodeQueryRequest request, CancellableTask task, ShardToQuery shardToQuery, AtomicInteger shardIndex, @@ -1035,10 +1040,10 @@ private static void executeOne( shardIndex.getAndIncrement(), shardToQuery.contextId, shardToQuery.originalIndices, - AliasFilter.EMPTY, + request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), null, shardToQuery.boost, - request, + request.searchRequest, 2, System.currentTimeMillis(), false @@ -1082,7 +1087,7 @@ private void maybeNext() { private static AbstractRunnable shardTask( SearchService searchService, - SearchRequest request, + NodeQueryRequest request, CancellableTask task, ShardToQuery shardToQuery, AtomicInteger shardIndex, @@ -1120,6 +1125,28 @@ public void onRejection(Exception e) { // minimum and ignore + requeue rejections in that case failures.put(shardToQuery.shardIndex, e); onDone.run(); + // TODO SO risk! + maybeNext(); + } + + private void maybeNext() { + var shardToQuery = shards.poll(); + if (shardToQuery != null) { + executor.execute( + shardTask( + searchService, + request, + task, + shardToQuery, + shardIndex, + shards, + executor, + queryPhaseResultConsumer, + failures, + onDone + ) + ); + } } }; } From 78067c6ab4527992a3422aa8a48948c07db1b998 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 11 Dec 2024 11:40:36 +0100 Subject: [PATCH 037/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 114 ++++++++---------- 1 file changed, 51 insertions(+), 63 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 38c9903123e62..3ac0a9b36be59 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -968,8 +968,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo try { final Object[] results = new Object[request.shards.size()]; for (int i = 0; i < results.length; i++) { - int shardIdx = request.shards.get(i).shardIndex; - var e = failures.get(shardIdx); + var e = failures.get(i); if (e != null) { results[i] = e; } else { @@ -1034,55 +1033,7 @@ private static void executeOne( Map failures, Runnable onDone ) { - final ShardSearchRequest req = buildShardSearchRequest( - shardToQuery.shardId, - null, - shardIndex.getAndIncrement(), - shardToQuery.contextId, - shardToQuery.originalIndices, - request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), - null, - shardToQuery.boost, - request.searchRequest, - 2, - System.currentTimeMillis(), - false - ); - searchService.executeQueryPhase(req, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult searchPhaseResult) { - searchPhaseResult.setShardIndex(req.shardRequestIndex()); - queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); - maybeNext(); - } - @Override - public void onFailure(Exception e) { - failures.put(shardToQuery.shardIndex, e); - onDone.run(); - maybeNext(); - } - - private void maybeNext() { - var shardToQuery = shards.poll(); - if (shardToQuery != null) { - executor.execute( - shardTask( - searchService, - request, - task, - shardToQuery, - shardIndex, - shards, - executor, - queryPhaseResultConsumer, - failures, - onDone - ) - ); - } - } - }); } private static AbstractRunnable shardTask( @@ -1097,21 +1048,58 @@ private static AbstractRunnable shardTask( Map failures, Runnable onDone ) { + final ShardSearchRequest req = buildShardSearchRequest( + shardToQuery.shardId, + null, + shardIndex.getAndIncrement(), + shardToQuery.contextId, + shardToQuery.originalIndices, + request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), + null, + shardToQuery.boost, + request.searchRequest, + 2, + System.currentTimeMillis(), + false + ); return new AbstractRunnable() { @Override protected void doRun() { - executeOne( - searchService, - request, - task, - shardToQuery, - shardIndex, - shards, - executor, - queryPhaseResultConsumer, - failures, - onDone - ); + searchService.executeQueryPhase(req, task, new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + searchPhaseResult.setShardIndex(req.shardRequestIndex()); + queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); + maybeNext(); + } + + @Override + public void onFailure(Exception e) { + failures.put(req.shardRequestIndex(), e); + onDone.run(); + maybeNext(); + } + + private void maybeNext() { + var shardToQuery = shards.poll(); + if (shardToQuery != null) { + executor.execute( + shardTask( + searchService, + request, + task, + shardToQuery, + shardIndex, + shards, + executor, + queryPhaseResultConsumer, + failures, + onDone + ) + ); + } + } + }); } @Override @@ -1123,7 +1111,7 @@ public void onFailure(Exception e) { public void onRejection(Exception e) { // TODO this could be done better now, we probably should only make sure to have a single loop running at // minimum and ignore + requeue rejections in that case - failures.put(shardToQuery.shardIndex, e); + failures.put(req.shardRequestIndex(), e); onDone.run(); // TODO SO risk! maybeNext(); From 1c8b9e70a9482c6355c20d6eb5d79d618f4feeff Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 11 Dec 2024 16:33:19 +0100 Subject: [PATCH 038/132] bck --- .../action/IndicesRequestIT.java | 11 ++---- .../admin/cluster/node/tasks/TasksIT.java | 5 +++ .../SearchQueryThenFetchAsyncAction.java | 35 ++++++------------- 3 files changed, 18 insertions(+), 33 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java index f5860cedcd989..3d4db2054935f 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/IndicesRequestIT.java @@ -571,11 +571,8 @@ public void testSearchQueryThenFetch() throws Exception { ); clearInterceptedActions(); - assertIndicesSubset( - Arrays.asList(searchRequest.indices()), - SearchTransportService.QUERY_ACTION_NAME, - SearchTransportService.FETCH_ID_ACTION_NAME - ); + assertIndicesSubset(Arrays.asList(searchRequest.indices()), true, SearchTransportService.QUERY_ACTION_NAME); + assertIndicesSubset(Arrays.asList(searchRequest.indices()), SearchTransportService.FETCH_ID_ACTION_NAME); } public void testSearchDfsQueryThenFetch() throws Exception { @@ -628,10 +625,6 @@ private static void assertIndicesSubset(List indices, String... actions) assertIndicesSubset(indices, false, actions); } - private static void assertIndicesSubsetOptionalRequests(List indices, String... actions) { - assertIndicesSubset(indices, true, actions); - } - private static void assertIndicesSubset(List indices, boolean optional, String... actions) { // indices returned by each bulk shard request need to be a subset of the original indices for (String action : actions) { diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index 1e16357a24412..4b666bbe42288 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction; import org.elasticsearch.action.bulk.TransportBulkAction; import org.elasticsearch.action.index.TransportIndexAction; +import org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction; import org.elasticsearch.action.search.SearchTransportService; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.action.support.WriteRequest; @@ -393,6 +394,10 @@ public void testSearchTaskDescriptions() { taskInfo.description(), Regex.simpleMatch("id[*], size[1], lastEmittedDoc[null]", taskInfo.description()) ); + case SearchQueryThenFetchAsyncAction.NODE_SEARCH_ACTION_NAME -> assertTrue( + taskInfo.description(), + Regex.simpleMatch("NodeQueryRequest", taskInfo.description()) + ); default -> fail("Unexpected action [" + taskInfo.action() + "] with description [" + taskInfo.description() + "]"); } // assert that all task descriptions have non-zero length diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 3ac0a9b36be59..c92e702cb92a4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -84,7 +84,7 @@ import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; import static org.elasticsearch.core.Strings.format; -class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearchContext { +public class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearchContext { private final Logger logger; private final NamedWriteableRegistry namedWriteableRegistry; @@ -381,7 +381,7 @@ private NodeQueryRequest(StreamInput in) throws IOException { @Override public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { - return new SearchShardTask(id, type, action, "", parentTaskId, headers); + return new SearchShardTask(id, type, action, "NodeQueryRequest", parentTaskId, headers); } @Override @@ -532,7 +532,7 @@ public void run() throws IOException { getOriginalIndices(shardIndex), shardIndex, routing.getShardId(), - shardsIts.get(shardIndex).getSearchContextId() + shardIterators[shardIndex].getSearchContextId() ) ); } else { @@ -1021,21 +1021,6 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo } - private static void executeOne( - SearchService searchService, - NodeQueryRequest request, - CancellableTask task, - ShardToQuery shardToQuery, - AtomicInteger shardIndex, - BlockingQueue shards, - Executor executor, - QueryPhaseResultConsumer queryPhaseResultConsumer, - Map failures, - Runnable onDone - ) { - - } - private static AbstractRunnable shardTask( SearchService searchService, NodeQueryRequest request, @@ -1048,6 +1033,7 @@ private static AbstractRunnable shardTask( Map failures, Runnable onDone ) { + var pitBuilder = request.searchRequest.pointInTimeBuilder(); final ShardSearchRequest req = buildShardSearchRequest( shardToQuery.shardId, null, @@ -1055,7 +1041,7 @@ private static AbstractRunnable shardTask( shardToQuery.contextId, shardToQuery.originalIndices, request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), - null, + pitBuilder == null ? null : pitBuilder.getKeepAlive(), shardToQuery.boost, request.searchRequest, 2, @@ -1104,11 +1090,6 @@ private void maybeNext() { @Override public void onFailure(Exception e) { - throw new AssertionError("shouldn't throw", e); - } - - @Override - public void onRejection(Exception e) { // TODO this could be done better now, we probably should only make sure to have a single loop running at // minimum and ignore + requeue rejections in that case failures.put(req.shardRequestIndex(), e); @@ -1117,6 +1098,12 @@ public void onRejection(Exception e) { maybeNext(); } + @Override + public void onRejection(Exception e) { + // TODO this could be done better now, we probably should only make sure to have a single loop running at + onFailure(e); + } + private void maybeNext() { var shardToQuery = shards.poll(); if (shardToQuery != null) { From 26f845e833b5c19056bff960b41d8a38d51848b3 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 11 Dec 2024 18:52:44 +0100 Subject: [PATCH 039/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index c92e702cb92a4..4b36f9ed60413 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -522,7 +522,8 @@ public void run() throws IOException { if (routing == null) { failOnUnavailable(shardIndex, shardRoutings); } else { - if (routing.getClusterAlias() == null && minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { + if ((routing.getClusterAlias() == null || Objects.equals(request.getLocalClusterAlias(), routing.getClusterAlias())) + && minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { perNodeQueries.computeIfAbsent( routing.getNodeId(), ignored -> new NodeQueryRequest(new ArrayList<>(), request, aliasFilter) @@ -546,7 +547,7 @@ public void run() throws IOException { this.performPhaseOnShard( shard.shardIndex, shardIterators[shard.shardIndex], - new SearchShardTarget(nodeId, shard.shardId, null) + new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()) ); return; } @@ -554,7 +555,7 @@ public void run() throws IOException { try { searchTransportService.transportService() .sendChildRequest( - getConnection(null, nodeId), + getConnection(request.searchRequest.getLocalClusterAlias(), nodeId), NODE_SEARCH_ACTION_NAME, request, task, @@ -578,7 +579,7 @@ public void handleResponse(NodeQueryResponse response) { if (response.results[i] instanceof Exception e) { onShardFailure( shardIdx, - new SearchShardTarget(nodeId, s.shardId(), null), + new SearchShardTarget(nodeId, s.shardId(), request.searchRequest.getLocalClusterAlias()), shardIterators[shardIdx], e ); @@ -586,7 +587,9 @@ public void handleResponse(NodeQueryResponse response) { } else if (response.results[i] instanceof QuerySearchResult q) { q.setShardIndex(shardIdx); var shardId = s.shardId(); - q.setSearchShardTarget(new SearchShardTarget(nodeId, shardId, null)); + q.setSearchShardTarget( + new SearchShardTarget(nodeId, shardId, request.searchRequest.getLocalClusterAlias()) + ); } } final int successfulShards = request.shards.size() - failedShards; @@ -607,14 +610,22 @@ public void handleResponse(NodeQueryResponse response) { @Override public void handleException(TransportException e) { for (ShardToQuery shard : request.shards) { - onShardFailure(shard.shardIndex, new SearchShardTarget(nodeId, shard.shardId, null), e); + onShardFailure( + shard.shardIndex, + new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), + e + ); } } } ); } catch (Exception e) { for (ShardToQuery shard : request.shards) { - onShardFailure(shard.shardIndex, new SearchShardTarget(nodeId, shard.shardId, null), e); + onShardFailure( + shard.shardIndex, + new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), + e + ); } } }); From 0b603e53df1a07a0d77fdec453260d97e41fe823 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 11 Dec 2024 21:58:04 +0100 Subject: [PATCH 040/132] bck --- .../action/search/SearchQueryThenFetchAsyncAction.java | 9 ++++----- .../test/hamcrest/ElasticsearchAssertions.java | 7 ++++++- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 4b36f9ed60413..8fbcff3a875cb 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -597,13 +597,12 @@ public void handleResponse(NodeQueryResponse response) { hasShardResponse.set(true); for (Object result : response.results) { if (result instanceof SearchPhaseResult searchPhaseResult) { - results.consumeResult( - searchPhaseResult, - () -> finishShardAndMaybePhase(searchPhaseResult.getSearchShardTarget().getShardId()) - ); + results.consumeResult(searchPhaseResult, () -> { + successfulOps.incrementAndGet(); + finishShardAndMaybePhase(searchPhaseResult.getSearchShardTarget().getShardId()); + }); } } - successfulOps.addAndGet(successfulShards); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java index 6c501898d5fe1..8aed66e173c7e 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java +++ b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java @@ -465,7 +465,12 @@ public static void assertScrollResponsesAndHitCount( public static void assertResponse(ActionFuture responseFuture, Consumer consumer) throws ExecutionException, InterruptedException { - var res = responseFuture.get(); + final R res; + try { + res = responseFuture.get(); + } catch (Exception e) { + throw new AssertionError(e); + } try { consumer.accept(res); } finally { From 5c214b619e5c874ace1d8d071a228f1f62252cb2 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 18 Dec 2024 17:23:08 +0100 Subject: [PATCH 041/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 3 +- .../action/search/SearchTransportService.java | 15 +++++----- .../elasticsearch/search/SearchService.java | 28 ++++++++++--------- .../search/rank/RankSearchContext.java | 3 +- 4 files changed, 25 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 8fbcff3a875cb..67c81675660b3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -24,7 +24,6 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; -import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; @@ -965,7 +964,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( request.searchRequest, executor, - new NoopCircuitBreaker(""), + searchService.circuitBreaker(), new SearchPhaseController(searchService::aggReduceContextBuilder), ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index b1875e4c3916e..4eacba2398891 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -45,6 +45,7 @@ import org.elasticsearch.search.query.ScrollQuerySearchResult; import org.elasticsearch.search.rank.feature.RankFeatureResult; import org.elasticsearch.search.rank.feature.RankFeatureShardRequest; +import org.elasticsearch.tasks.CancellableTask; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteClusterService; @@ -433,7 +434,7 @@ public static void registerRequestHandler(SearchTransportService searchTransport DFS_ACTION_NAME, EsExecutors.DIRECT_EXECUTOR_SERVICE, ShardSearchRequest::new, - (request, channel, task) -> searchService.executeDfsPhase(request, (SearchShardTask) task, new ChannelActionListener<>(channel)) + (request, channel, task) -> searchService.executeDfsPhase(request, (CancellableTask) task, new ChannelActionListener<>(channel)) ); TransportActionProxy.registerProxyAction(transportService, DFS_ACTION_NAME, true, DfsSearchResult::new); @@ -443,7 +444,7 @@ public static void registerRequestHandler(SearchTransportService searchTransport ShardSearchRequest::new, (request, channel, task) -> searchService.executeQueryPhase( request, - (SearchShardTask) task, + (CancellableTask) task, new ChannelActionListener<>(channel) ) ); @@ -460,7 +461,7 @@ public static void registerRequestHandler(SearchTransportService searchTransport QuerySearchRequest::new, (request, channel, task) -> searchService.executeQueryPhase( request, - (SearchShardTask) task, + (CancellableTask) task, new ChannelActionListener<>(channel), channel.getVersion() ) @@ -473,7 +474,7 @@ public static void registerRequestHandler(SearchTransportService searchTransport InternalScrollSearchRequest::new, (request, channel, task) -> searchService.executeQueryPhase( request, - (SearchShardTask) task, + (CancellableTask) task, new ChannelActionListener<>(channel), channel.getVersion() ) @@ -486,14 +487,14 @@ public static void registerRequestHandler(SearchTransportService searchTransport InternalScrollSearchRequest::new, (request, channel, task) -> searchService.executeFetchPhase( request, - (SearchShardTask) task, + (CancellableTask) task, new ChannelActionListener<>(channel) ) ); TransportActionProxy.registerProxyAction(transportService, QUERY_FETCH_SCROLL_ACTION_NAME, true, ScrollQueryFetchSearchResult::new); final TransportRequestHandler rankShardFeatureRequest = (request, channel, task) -> searchService - .executeRankFeaturePhase(request, (SearchShardTask) task, new ChannelActionListener<>(channel)); + .executeRankFeaturePhase(request, (CancellableTask) task, new ChannelActionListener<>(channel)); transportService.registerRequestHandler( RANK_FEATURE_SHARD_ACTION_NAME, EsExecutors.DIRECT_EXECUTOR_SERVICE, @@ -503,7 +504,7 @@ public static void registerRequestHandler(SearchTransportService searchTransport TransportActionProxy.registerProxyAction(transportService, RANK_FEATURE_SHARD_ACTION_NAME, true, RankFeatureResult::new); final TransportRequestHandler shardFetchRequestHandler = (request, channel, task) -> searchService - .executeFetchPhase(request, (SearchShardTask) task, new ChannelActionListener<>(channel)); + .executeFetchPhase(request, (CancellableTask) task, new ChannelActionListener<>(channel)); transportService.registerRequestHandler( FETCH_ID_SCROLL_ACTION_NAME, EsExecutors.DIRECT_EXECUTOR_SERVICE, diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index ed026e005041c..a1887eb5df09a 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -24,7 +24,6 @@ import org.elasticsearch.action.ResolvedIndices; import org.elasticsearch.action.search.CanMatchNodeRequest; import org.elasticsearch.action.search.CanMatchNodeResponse; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.ClusterState; @@ -323,6 +322,8 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv private final Tracer tracer; + private final CircuitBreaker circuitBreaker; + public SearchService( ClusterService clusterService, IndicesService indicesService, @@ -341,11 +342,8 @@ public SearchService( this.scriptService = scriptService; this.bigArrays = bigArrays; this.fetchPhase = fetchPhase; - this.multiBucketConsumerService = new MultiBucketConsumerService( - clusterService, - settings, - circuitBreakerService.getBreaker(CircuitBreaker.REQUEST) - ); + this.circuitBreaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST); + this.multiBucketConsumerService = new MultiBucketConsumerService(clusterService, settings, circuitBreaker); this.executorSelector = executorSelector; this.tracer = tracer; @@ -392,6 +390,10 @@ public SearchService( .addSettingsUpdateConsumer(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED, this::setEnableQueryPhaseParallelCollection); } + public CircuitBreaker circuitBreaker() { + return circuitBreaker; + } + private void setEnableSearchWorkerThreads(boolean enableSearchWorkerThreads) { if (enableSearchWorkerThreads) { searchExecutor = threadPool.executor(Names.SEARCH); @@ -544,7 +546,7 @@ static ActionListener maybeWrapListenerForStackTrace( return listener; } - public void executeDfsPhase(ShardSearchRequest request, SearchShardTask task, ActionListener listener) { + public void executeDfsPhase(ShardSearchRequest request, CancellableTask task, ActionListener listener) { listener = maybeWrapListenerForStackTrace(listener, request.getChannelVersion(), threadPool); final IndexShard shard = getShard(request); rewriteAndFetchShardRequest(shard, request, listener.delegateFailure((l, rewritten) -> { @@ -553,7 +555,7 @@ public void executeDfsPhase(ShardSearchRequest request, SearchShardTask task, Ac })); } - private DfsSearchResult executeDfsPhase(ShardSearchRequest request, SearchShardTask task) throws IOException { + private DfsSearchResult executeDfsPhase(ShardSearchRequest request, CancellableTask task) throws IOException { ReaderContext readerContext = createOrGetReaderContext(request); try (@SuppressWarnings("unused") // withScope call is necessary to instrument search execution Releasable scope = tracer.withScope(task); @@ -775,7 +777,7 @@ private SearchPhaseResult executeQueryPhase(ShardSearchRequest request, Cancella } } - public void executeRankFeaturePhase(RankFeatureShardRequest request, SearchShardTask task, ActionListener listener) { + public void executeRankFeaturePhase(RankFeatureShardRequest request, CancellableTask task, ActionListener listener) { listener = maybeWrapListenerForStackTrace(listener, request.getShardSearchRequest().getChannelVersion(), threadPool); final ReaderContext readerContext = findReaderContext(request.contextId(), request); final ShardSearchRequest shardSearchRequest = readerContext.getShardSearchRequest(request.getShardSearchRequest()); @@ -819,7 +821,7 @@ private QueryFetchSearchResult executeFetchPhase(ReaderContext reader, SearchCon public void executeQueryPhase( InternalScrollSearchRequest request, - SearchShardTask task, + CancellableTask task, ActionListener listener, TransportVersion version ) { @@ -861,7 +863,7 @@ public void executeQueryPhase( */ public void executeQueryPhase( QuerySearchRequest request, - SearchShardTask task, + CancellableTask task, ActionListener listener, TransportVersion version ) { @@ -919,7 +921,7 @@ private Executor getExecutor(IndexShard indexShard) { public void executeFetchPhase( InternalScrollSearchRequest request, - SearchShardTask task, + CancellableTask task, ActionListener listener ) { final LegacyReaderContext readerContext = (LegacyReaderContext) findReaderContext(request.contextId(), request); @@ -954,7 +956,7 @@ public void executeFetchPhase( }, wrapFailureListener(listener, readerContext, markAsUsed)); } - public void executeFetchPhase(ShardFetchRequest request, SearchShardTask task, ActionListener listener) { + public void executeFetchPhase(ShardFetchRequest request, CancellableTask task, ActionListener listener) { final ReaderContext readerContext = findReaderContext(request.contextId(), request); final ShardSearchRequest shardSearchRequest = readerContext.getShardSearchRequest(request.getShardSearchRequest()); final Releasable markAsUsed = readerContext.markAsUsed(getKeepAlive(shardSearchRequest)); diff --git a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java index 2f09e3404d551..951a9b0cf3520 100644 --- a/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java +++ b/server/src/main/java/org/elasticsearch/search/rank/RankSearchContext.java @@ -12,7 +12,6 @@ import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.Query; import org.apache.lucene.search.TotalHits; -import org.elasticsearch.action.search.SearchShardTask; import org.elasticsearch.action.search.SearchType; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.cache.bitset.BitsetFilterCache; @@ -217,7 +216,7 @@ public void setTask(CancellableTask task) { } @Override - public SearchShardTask getTask() { + public CancellableTask getTask() { throw new UnsupportedOperationException(); } From 4c7bc8208f1a4342874996ec405a742195c99a84 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 18 Dec 2024 22:27:41 +0100 Subject: [PATCH 042/132] fix but also breakage --- .../elasticsearch/action/ActionListener.java | 13 +++-- .../search/AbstractSearchAsyncAction.java | 5 +- .../search/QueryPhaseResultConsumer.java | 11 +++-- .../SearchQueryThenFetchAsyncAction.java | 48 ++++++++++++------- .../action/search/SearchTransportService.java | 17 +++++-- .../action/search/TransportSearchAction.java | 2 +- .../elasticsearch/search/SearchService.java | 10 +++- 7 files changed, 76 insertions(+), 30 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ActionListener.java b/server/src/main/java/org/elasticsearch/action/ActionListener.java index a158669d936fe..79d0d29a591f1 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListener.java @@ -388,14 +388,19 @@ static ActionListener assertOnce(ActionListener d // if complete, records the stack trace which first completed it private final AtomicReference firstCompletion = new AtomicReference<>(); - private void assertFirstRun() { + private void assertFirstRun(Exception t) { var previousRun = firstCompletion.compareAndExchange(null, new ElasticsearchException("executed already")); - assert previousRun == null : "[" + delegate + "] " + previousRun; // reports the stack traces of both completions + // reports the stack traces of both completions + assert previousRun == null + : new AssertionError( + "[" + delegate + "] " + previousRun, + t == null ? previousRun : ExceptionsHelper.useOrSuppress(previousRun, t) + ); } @Override public void onResponse(Response response) { - assertFirstRun(); + assertFirstRun(null); try { delegate.onResponse(response); } catch (Exception e) { @@ -406,7 +411,7 @@ public void onResponse(Response response) { @Override public void onFailure(Exception e) { - assertFirstRun(); + assertFirstRun(e); safeOnFailure(delegate, e); } diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 1ba3cd0cc6b75..f041d936b73da 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -441,7 +441,10 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final if (request.allowPartialSearchResults() == false) { if (requestCancelled.compareAndSet(false, true)) { try { - searchTransportService.cancelSearchTask(task, "partial results are not allowed and at least one shard has failed"); + searchTransportService.cancelSearchTask( + task.getId(), + "partial results are not allowed and at least one shard has failed" + ); } catch (Exception cancelFailure) { logger.debug("Failed to cancel search request", cancelFailure); } diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 35309bd36e1bd..d5c57520151b8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -86,7 +86,7 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults queue = new ArrayDeque<>(); private final AtomicReference runningTask = new AtomicReference<>(); - private final AtomicReference failure = new AtomicReference<>(); + public final AtomicReference failure = new AtomicReference<>(); public final TopDocsStats topDocsStats; private volatile MergeResult mergeResult; @@ -124,7 +124,12 @@ public QueryPhaseResultConsumer( this.hasTopDocs = (source == null || size != 0) && queryPhaseRankCoordinatorContext == null; this.hasAggs = source != null && source.aggregations() != null; this.aggReduceContextBuilder = hasAggs ? controller.getReduceContext(isCanceled, source.aggregations()) : null; - batchReduceSize = (hasAggs || hasTopDocs) ? Math.min(request.getBatchedReduceSize(), expectedResultSize) : expectedResultSize; + // TODO: facepalm + if (request.getBatchedReduceSize() == Integer.MAX_VALUE) { + batchReduceSize = Integer.MAX_VALUE; + } else { + batchReduceSize = (hasAggs || hasTopDocs) ? Math.min(request.getBatchedReduceSize(), expectedResultSize) : expectedResultSize; + } topDocsStats = new TopDocsStats(request.resolveTrackTotalHitsUpTo()); } @@ -332,7 +337,7 @@ public int getNumReducePhases() { return numReducePhases; } - private boolean hasFailure() { + public boolean hasFailure() { return failure.get() != null; } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 67c81675660b3..7d3a6a0604772 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -611,6 +611,7 @@ public void handleException(TransportException e) { onShardFailure( shard.shardIndex, new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), + shardsIts.get(shard.shardIndex), e ); } @@ -710,7 +711,10 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final if (request.allowPartialSearchResults() == false) { if (requestCancelled.compareAndSet(false, true)) { try { - searchTransportService.cancelSearchTask(task, "partial results are not allowed and at least one shard has failed"); + searchTransportService.cancelSearchTask( + task.getId(), + "partial results are not allowed and at least one shard has failed" + ); } catch (Exception cancelFailure) { logger.debug("Failed to cancel search request", cancelFailure); } @@ -730,7 +734,7 @@ private void finishShardAndMaybePhase(ShardId shardId) { boolean removed = outstandingShards.remove(shardId); assert removed : "unknown shardId " + shardId; if (outstandingShards.isEmpty()) { - onPhaseDone(); + executeNextPhase(this, this::getNextPhase); } } @@ -791,10 +795,6 @@ private void successfulShardExecution(SearchShardIterator shardsIt) { finishShardAndMaybePhase(shardsIt.shardId()); } - final void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() - executeNextPhase(this, this::getNextPhase); - } - @Override public void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { /* This is the main search phase transition where we move to the next phase. If all shards @@ -961,6 +961,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo final AtomicInteger shardIndex = new AtomicInteger(); request.searchRequest.finalReduce = false; request.searchRequest.setBatchedReduceSize(Integer.MAX_VALUE); + final CountDown countDown = new CountDown(request.shards.size()); final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( request.searchRequest, executor, @@ -969,12 +970,17 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, request.shards.size(), - e -> searchTransportService.cancelSearchTask((SearchTask) task, "failed to merge result [" + e.getMessage() + "]") + e -> {} ); - final CountDown countDown = new CountDown(request.shards.size()); final Runnable onDone = () -> { if (countDown.countDown()) { + var channelListener = new ChannelActionListener<>(channel); try { + var failure = queryPhaseResultConsumer.failure.get(); + if (failure != null) { + channelListener.onFailure(failure); + return; + } final Object[] results = new Object[request.shards.size()]; for (int i = 0; i < results.length; i++) { var e = failures.get(i); @@ -987,7 +993,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo } // TODO: facepalm queryPhaseResultConsumer.buffer.clear(); - new ChannelActionListener<>(channel).onResponse( + channelListener.onResponse( new NodeQueryResponse( new QueryPhaseResultConsumer.MergeResult( request.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), @@ -999,7 +1005,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo queryPhaseResultConsumer.topDocsStats ) ); - } catch (Exception e) { + } catch (Throwable e) { throw new AssertionError(e); } finally { queryPhaseResultConsumer.close(); @@ -1063,16 +1069,26 @@ protected void doRun() { searchService.executeQueryPhase(req, task, new ActionListener<>() { @Override public void onResponse(SearchPhaseResult searchPhaseResult) { - searchPhaseResult.setShardIndex(req.shardRequestIndex()); - queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); - maybeNext(); + try { + searchPhaseResult.setShardIndex(req.shardRequestIndex()); + queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); + } catch (Throwable e) { + throw new AssertionError(e); + } finally { + maybeNext(); + } } @Override public void onFailure(Exception e) { - failures.put(req.shardRequestIndex(), e); - onDone.run(); - maybeNext(); + try { + failures.put(req.shardRequestIndex(), e); + onDone.run(); + maybeNext(); + } catch (Throwable expected) { + expected.addSuppressed(e); + throw new AssertionError(expected); + } } private void maybeNext() { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index 4eacba2398891..d8d1265b62bd1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -16,6 +16,7 @@ import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; import org.elasticsearch.action.admin.cluster.node.tasks.get.TransportGetTaskAction; +import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; import org.elasticsearch.action.support.ChannelActionListener; import org.elasticsearch.client.internal.OriginSettingClient; import org.elasticsearch.client.internal.node.NodeClient; @@ -617,10 +618,20 @@ private boolean assertNodePresent() { } } - public void cancelSearchTask(SearchTask task, String reason) { - CancelTasksRequest req = new CancelTasksRequest().setTargetTaskId(new TaskId(client.getLocalNodeId(), task.getId())) + public void cancelSearchTask(long taskId, String reason) { + CancelTasksRequest req = new CancelTasksRequest().setTargetTaskId(new TaskId(client.getLocalNodeId(), taskId)) .setReason("Fatal failure during search: " + reason); // force the origin to execute the cancellation as a system user - new OriginSettingClient(client, TransportGetTaskAction.TASKS_ORIGIN).admin().cluster().cancelTasks(req, ActionListener.noop()); + new OriginSettingClient(client, TransportGetTaskAction.TASKS_ORIGIN).admin().cluster().cancelTasks(req, new ActionListener<>() { + @Override + public void onResponse(ListTasksResponse listTasksResponse) { + + } + + @Override + public void onFailure(Exception e) { + logger.warn("unexpected failure cancelling [" + taskId + "] because of [" + reason + "]", e); + } + }); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index b166b49194efa..26dd880ce8d96 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -1519,7 +1519,7 @@ public void runNewSearchPhase( task.getProgressListener(), searchRequest, shardIterators.size(), - exc -> searchTransportService.cancelSearchTask(task, "failed to merge result [" + exc.getMessage() + "]") + exc -> searchTransportService.cancelSearchTask(task.getId(), "failed to merge result [" + exc.getMessage() + "]") ); boolean success = false; try { diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index a1887eb5df09a..b0929c2584441 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -603,7 +603,7 @@ public void executeQueryPhase(ShardSearchRequest request, CancellableTask task, ); CanMatchShardResponse canMatchResp = canMatch(canMatchContext, false); if (canMatchResp.canMatch() == false) { - finalListener.onResponse(QuerySearchResult.nullInstance()); + l.onResponse(QuerySearchResult.nullInstance()); return; } } @@ -957,7 +957,13 @@ public void executeFetchPhase( } public void executeFetchPhase(ShardFetchRequest request, CancellableTask task, ActionListener listener) { - final ReaderContext readerContext = findReaderContext(request.contextId(), request); + final ReaderContext readerContext; + try { + readerContext = findReaderContext(request.contextId(), request); + } catch (Exception e) { + listener.onFailure(e); + return; + } final ShardSearchRequest shardSearchRequest = readerContext.getShardSearchRequest(request.getShardSearchRequest()); final Releasable markAsUsed = readerContext.markAsUsed(getKeepAlive(shardSearchRequest)); runAsync(getExecutor(readerContext.indexShard()), () -> { From c61eaae98da2ecbd891680ecdbb837e807c0a78c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 19 Dec 2024 19:06:57 +0100 Subject: [PATCH 043/132] bck --- .../SearchQueryThenFetchAsyncAction.java | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 7d3a6a0604772..23a424765a758 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -364,11 +364,18 @@ public static class NodeQueryRequest extends TransportRequest { private final List shards; private final SearchRequest searchRequest; private final Map aliasFilters; + private final int totalShards; - private NodeQueryRequest(List shards, SearchRequest searchRequest, Map aliasFilters) { + private NodeQueryRequest( + List shards, + SearchRequest searchRequest, + Map aliasFilters, + int totalShards + ) { this.shards = shards; this.searchRequest = searchRequest; this.aliasFilters = aliasFilters; + this.totalShards = totalShards; } private NodeQueryRequest(StreamInput in) throws IOException { @@ -376,6 +383,7 @@ private NodeQueryRequest(StreamInput in) throws IOException { this.shards = in.readCollectionAsImmutableList(ShardToQuery::readFrom); this.searchRequest = new SearchRequest(in); this.aliasFilters = in.readImmutableMap(AliasFilter::readFrom); + this.totalShards = in.readVInt(); } @Override @@ -389,6 +397,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeCollection(shards); searchRequest.writeTo(out); out.writeMap(aliasFilters, (o, v) -> v.writeTo(o)); + out.writeVInt(totalShards); } } @@ -525,7 +534,7 @@ public void run() throws IOException { && minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { perNodeQueries.computeIfAbsent( routing.getNodeId(), - ignored -> new NodeQueryRequest(new ArrayList<>(), request, aliasFilter) + ignored -> new NodeQueryRequest(new ArrayList<>(), request, aliasFilter, shardsIts.size()) ).shards.add( new ShardToQuery( concreteIndexBoosts.getOrDefault(routing.getShardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), @@ -1049,17 +1058,18 @@ private static AbstractRunnable shardTask( Runnable onDone ) { var pitBuilder = request.searchRequest.pointInTimeBuilder(); + final int dataNodeLocalIdx = shardIndex.getAndIncrement(); final ShardSearchRequest req = buildShardSearchRequest( shardToQuery.shardId, null, - shardIndex.getAndIncrement(), + shardToQuery.shardIndex, shardToQuery.contextId, shardToQuery.originalIndices, request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), pitBuilder == null ? null : pitBuilder.getKeepAlive(), shardToQuery.boost, request.searchRequest, - 2, + request.totalShards, System.currentTimeMillis(), false ); @@ -1070,7 +1080,7 @@ protected void doRun() { @Override public void onResponse(SearchPhaseResult searchPhaseResult) { try { - searchPhaseResult.setShardIndex(req.shardRequestIndex()); + searchPhaseResult.setShardIndex(dataNodeLocalIdx); queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); } catch (Throwable e) { throw new AssertionError(e); @@ -1082,7 +1092,7 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { @Override public void onFailure(Exception e) { try { - failures.put(req.shardRequestIndex(), e); + failures.put(dataNodeLocalIdx, e); onDone.run(); maybeNext(); } catch (Throwable expected) { @@ -1117,7 +1127,7 @@ private void maybeNext() { public void onFailure(Exception e) { // TODO this could be done better now, we probably should only make sure to have a single loop running at // minimum and ignore + requeue rejections in that case - failures.put(req.shardRequestIndex(), e); + failures.put(dataNodeLocalIdx, e); onDone.run(); // TODO SO risk! maybeNext(); From d3cbc0540ae9a8dc5fda41ada3725a92b9d395a6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 20 Dec 2024 15:47:11 +0100 Subject: [PATCH 044/132] cleanup --- .../search/SearchQueryThenFetchAsyncAction.java | 14 +++++--------- .../action/search/SearchTransportService.java | 4 +++- .../action/search/TransportSearchAction.java | 1 - .../SearchQueryThenFetchAsyncActionTests.java | 1 - 4 files changed, 8 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 23a424765a758..e42f95bc2d47d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.util.SetOnce; @@ -85,7 +86,7 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearchContext { - private final Logger logger; + private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class); private final NamedWriteableRegistry namedWriteableRegistry; private final SearchTransportService searchTransportService; private final Executor executor; @@ -129,7 +130,6 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn private final Client client; SearchQueryThenFetchAsyncAction( - Logger logger, NamedWriteableRegistry namedWriteableRegistry, SearchTransportService searchTransportService, BiFunction nodeIdToConnection, @@ -166,7 +166,6 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn // consistent between two requests that target the same shards. Arrays.sort(shardIterators); this.timeProvider = timeProvider; - this.logger = logger; this.searchTransportService = searchTransportService; this.executor = executor; this.request = request; @@ -617,12 +616,8 @@ public void handleResponse(NodeQueryResponse response) { @Override public void handleException(TransportException e) { for (ShardToQuery shard : request.shards) { - onShardFailure( - shard.shardIndex, - new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), - shardsIts.get(shard.shardIndex), - e - ); + var shardIt = shardsIts.get(shard.shardIndex); + onShardFailure(shard.shardIndex, shardIt.current, shardIt, e); } } } @@ -923,6 +918,7 @@ private void raisePhaseFailure(SearchPhaseExecutionException exception) { listener.onFailure(exception); } + @Override public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { assert isPartOfPointInTime(contextId) == false : "Must not release point in time context [" + contextId + "]"; if (connection != null) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java index d8d1265b62bd1..d6ae2d9ff05ed 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchTransportService.java @@ -396,6 +396,8 @@ public static void registerRequestHandler(SearchTransportService searchTransport transportService.registerRequestHandler( FREE_CONTEXT_SCROLL_ACTION_NAME, freeContextExecutor, + false, + false, ScrollFreeContextRequest::new, freeContextHandler ); @@ -407,7 +409,7 @@ public static void registerRequestHandler(SearchTransportService searchTransport ); // TODO: remove this handler once the lowest compatible version stops using it - transportService.registerRequestHandler(FREE_CONTEXT_ACTION_NAME, freeContextExecutor, in -> { + transportService.registerRequestHandler(FREE_CONTEXT_ACTION_NAME, freeContextExecutor, false, false, in -> { var res = new ScrollFreeContextRequest(in); // this handler exists for BwC purposes only, we don't need the original indices to free the context OriginalIndices.readOriginalIndices(in); diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 26dd880ce8d96..d0f03881b2b69 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -1547,7 +1547,6 @@ public void runNewSearchPhase( } else { assert searchRequest.searchType() == QUERY_THEN_FETCH : searchRequest.searchType(); var searchPhase = new SearchQueryThenFetchAsyncAction( - logger, namedWriteableRegistry, searchTransportService, connectionLookup, diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 6357155793fdf..f41925f90c825 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -186,7 +186,6 @@ public void sendExecuteQuery( ) ) { SearchQueryThenFetchAsyncAction action = new SearchQueryThenFetchAsyncAction( - logger, null, searchTransportService, (clusterAlias, node) -> lookup.get(node), From 1b454bc56dc594d2d2dbed433043642b7c0632b4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 20 Dec 2024 18:01:19 +0100 Subject: [PATCH 045/132] cleanup --- .../action/search/TransportSearchIT.java | 40 ++++++------------- .../SearchQueryThenFetchAsyncAction.java | 33 +++++++++++++-- 2 files changed, 43 insertions(+), 30 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java index 5db2651c703d2..d4d2fe4f6d7ff 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java @@ -73,10 +73,12 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; +import java.util.stream.IntStream; 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.assertResponses; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -449,33 +451,16 @@ public void testCircuitBreakerReduceFail() throws Exception { indexSomeDocs("test", numShards, numShards * 3); { - final AtomicArray responses = new AtomicArray<>(10); - final CountDownLatch latch = new CountDownLatch(10); - for (int i = 0; i < 10; i++) { - int batchReduceSize = randomIntBetween(2, Math.max(numShards + 1, 3)); - SearchRequest request = prepareSearch("test").addAggregation(new TestAggregationBuilder("test")) - .setBatchedReduceSize(batchReduceSize) - .request(); - final int index = i; - client().search(request, new ActionListener<>() { - @Override - public void onResponse(SearchResponse response) { - responses.set(index, true); - latch.countDown(); - } - - @Override - public void onFailure(Exception e) { - responses.set(index, false); - latch.countDown(); - } - }); - } - latch.await(); - assertThat(responses.asList().size(), equalTo(10)); - for (boolean resp : responses.asList()) { - assertTrue(resp); - } + assertResponses( + r -> {}, + IntStream.range(0, 10) + .map(i -> randomIntBetween(2, Math.max(numShards + 1, 3))) + .mapToObj( + batchReduceSize -> prepareSearch("test").addAggregation(new TestAggregationBuilder("test")) + .setBatchedReduceSize(batchReduceSize) + ) + .toArray(SearchRequestBuilder[]::new) + ); assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); } @@ -520,6 +505,7 @@ public void onFailure(Exception exc) { } finally { updateClusterSettings(Settings.builder().putNull("indices.breaker.request.limit")); } + logger.info("--> done"); } public void testCircuitBreakerFetchFail() throws Exception { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index e42f95bc2d47d..209cadb3a0041 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -25,6 +25,7 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.StreamInput; @@ -617,7 +618,12 @@ public void handleResponse(NodeQueryResponse response) { public void handleException(TransportException e) { for (ShardToQuery shard : request.shards) { var shardIt = shardsIts.get(shard.shardIndex); - onShardFailure(shard.shardIndex, shardIt.current, shardIt, e); + onShardFailure( + shard.shardIndex, + new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), + shardIt, + e + ); } } } @@ -970,7 +976,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( request.searchRequest, executor, - searchService.circuitBreaker(), + new NoopCircuitBreaker("request"), new SearchPhaseController(searchService::aggReduceContextBuilder), ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, @@ -983,16 +989,28 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo try { var failure = queryPhaseResultConsumer.failure.get(); if (failure != null) { + try { + queryPhaseResultConsumer.getSuccessfulResults().forEach(searchPhaseResult -> { + SearchPhaseResult phaseResult = searchPhaseResult.queryResult() != null + ? searchPhaseResult.queryResult() + : searchPhaseResult.rankFeatureResult(); + maybeRelease(searchService, request, phaseResult); + }); + } catch (Throwable e) { + throw new RuntimeException(e); + } channelListener.onFailure(failure); return; } final Object[] results = new Object[request.shards.size()]; for (int i = 0; i < results.length; i++) { var e = failures.get(i); + var res = queryPhaseResultConsumer.results.get(i); if (e != null) { results[i] = e; + assert res == null; } else { - results[i] = queryPhaseResultConsumer.results.get(i); + results[i] = res; assert results[i] != null; } } @@ -1041,6 +1059,15 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo } + private static void maybeRelease(SearchService searchService, NodeQueryRequest request, SearchPhaseResult phaseResult) { + if (phaseResult != null + && phaseResult.hasSearchContext() + && request.searchRequest.scroll() == null + && (AbstractSearchAsyncAction.isPartOfPIT(null, request.searchRequest, phaseResult.getContextId()) == false)) { + searchService.freeReaderContext(phaseResult.getContextId()); + } + } + private static AbstractRunnable shardTask( SearchService searchService, NodeQueryRequest request, From 6449f87dd99198fdf24b672f623b7f9f37126a2c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 20 Dec 2024 21:07:36 +0100 Subject: [PATCH 046/132] cleanup --- .../search/AbstractSearchAsyncAction.java | 22 ++++++++++--------- .../action/search/AsyncSearchContext.java | 2 +- .../action/search/DfsQueryPhase.java | 2 +- .../action/search/ExpandSearchPhase.java | 2 +- .../action/search/FetchLookupFieldsPhase.java | 4 ++-- .../action/search/FetchSearchPhase.java | 4 ++-- .../action/search/RankFeaturePhase.java | 6 ++--- .../SearchQueryThenFetchAsyncAction.java | 22 ++++++++++--------- .../AbstractSearchAsyncActionTests.java | 7 +----- .../action/search/MockSearchPhaseContext.java | 4 ++-- 10 files changed, 37 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index f041d936b73da..fca6f026f68ea 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -350,14 +350,15 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. */ + final String currentPhaseName = currentPhase.getName(); ShardOperationFailedException[] shardSearchFailures = buildShardFailures(shardFailures); if (shardSearchFailures.length == getNumShards()) { shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = shardSearchFailures.length == 0 ? null : ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> "All shards failed for phase: [" + currentPhase.getName() + "]", cause); - onPhaseFailure(currentPhase, "all shards failed", cause); + logger.debug(() -> "All shards failed for phase: [" + currentPhaseName + "]", cause); + onPhaseFailure(currentPhaseName, "all shards failed", cause); } else { Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; @@ -369,9 +370,9 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex int numShardFailures = shardSearchFailures.length; shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase.getName()), cause); + logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhaseName), cause); } - onPhaseFailure(currentPhase, "Partial shards failure", null); + onPhaseFailure(currentPhaseName, "Partial shards failure", null); } else { int discrepancy = getNumShards() - successfulOps.get(); assert discrepancy > 0 : "discrepancy: " + discrepancy; @@ -382,10 +383,10 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex successfulOps.get(), skippedOps.get(), getNumShards(), - currentPhase.getName() + currentPhaseName ); } - onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); + onPhaseFailure(currentPhaseName, "Partial shards failure (" + discrepancy + " shards unavailable)", null); } return; } @@ -396,7 +397,7 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex .collect(Collectors.joining(",")); logger.trace( "[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})", - currentPhase.getName(), + currentPhaseName, nextPhase.getName(), resultsFrom, clusterStateVersion @@ -413,7 +414,7 @@ private void executePhase(SearchPhase phase) { if (logger.isDebugEnabled()) { logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, phase.getName()), e); } - onPhaseFailure(phase, "", e); + onPhaseFailure(phase.getName(), "", e); } } @@ -701,8 +702,9 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At * @param msg an optional message * @param cause the cause of the phase failure */ - public void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { - raisePhaseFailure(new SearchPhaseExecutionException(phase.getName(), msg, cause, buildShardFailures(shardFailures))); + @Override + public void onPhaseFailure(String phase, String msg, Throwable cause) { + raisePhaseFailure(new SearchPhaseExecutionException(phase, msg, cause, buildShardFailures(shardFailures))); } /** diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 489631f760ce8..4f104cbf66402 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -29,7 +29,7 @@ public interface AsyncSearchContext { SearchTask getTask(); - void onPhaseFailure(SearchPhase phase, String msg, Throwable cause); + void onPhaseFailure(String phase, String msg, Throwable cause); void addReleasable(Releasable releasable); diff --git a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java index cc8c4becea9a9..7f1a29677b04e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java @@ -106,7 +106,7 @@ protected void innerOnResponse(QuerySearchResult response) { response.setSearchProfileDfsPhaseResult(dfsResult.searchProfileDfsPhaseResult()); counter.onResult(response); } catch (Exception e) { - context.onPhaseFailure(DfsQueryPhase.this, "", e); + context.onPhaseFailure(DfsQueryPhase.this.getName(), "", e); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java index 5935e630b0518..c2529be244dad 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java @@ -123,7 +123,7 @@ private void doRun() { } private void phaseFailure(Exception ex) { - context.onPhaseFailure(this, "failed to expand hits", ex); + context.onPhaseFailure(this.getName(), "failed to expand hits", ex); } private static SearchSourceBuilder buildExpandSearchSourceBuilder(InnerHitBuilder options, CollapseBuilder innerCollapseBuilder) { diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java index e2b157b602e05..d09a1b16dfd63 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java @@ -125,7 +125,7 @@ public void onResponse(MultiSearchResponse items) { } } if (failure != null) { - context.onPhaseFailure(FetchLookupFieldsPhase.this, "failed to fetch lookup fields", failure); + context.onPhaseFailure(FetchLookupFieldsPhase.this.getName(), "failed to fetch lookup fields", failure); } else { context.sendSearchResponse(searchResponse, queryResults); } @@ -133,7 +133,7 @@ public void onResponse(MultiSearchResponse items) { @Override public void onFailure(Exception e) { - context.onPhaseFailure(FetchLookupFieldsPhase.this, "failed to fetch lookup fields", e); + context.onPhaseFailure(FetchLookupFieldsPhase.this.getName(), "failed to fetch lookup fields", e); } }); } diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 7f88035f971d7..3c9d70fc9144a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -95,7 +95,7 @@ protected void doRun() throws Exception { @Override public void onFailure(Exception e) { - context.onPhaseFailure(FetchSearchPhase.this, "", e); + context.onPhaseFailure(FetchSearchPhase.this.getName(), "", e); } }); } @@ -216,7 +216,7 @@ public void innerOnResponse(FetchSearchResult result) { progressListener.notifyFetchResult(shardIndex); counter.onResult(result); } catch (Exception e) { - context.onPhaseFailure(FetchSearchPhase.this, "", e); + context.onPhaseFailure(FetchSearchPhase.this.getName(), "", e); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java index 6a359e659bf94..3ce3fce7ab376 100644 --- a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java @@ -76,7 +76,7 @@ protected void doRun() throws Exception { @Override public void onFailure(Exception e) { - context.onPhaseFailure(RankFeaturePhase.this, "", e); + context.onPhaseFailure(RankFeaturePhase.this.getName(), "", e); } }); } @@ -131,7 +131,7 @@ protected void innerOnResponse(RankFeatureResult response) { progressListener.notifyRankFeatureResult(shardIndex); rankRequestCounter.onResult(response); } catch (Exception e) { - context.onPhaseFailure(RankFeaturePhase.this, "", e); + context.onPhaseFailure(RankFeaturePhase.this.getName(), "", e); } } @@ -186,7 +186,7 @@ public void onResponse(RankFeatureDoc[] docsWithUpdatedScores) { @Override public void onFailure(Exception e) { - context.onPhaseFailure(RankFeaturePhase.this, "Computing updated ranks for results failed", e); + context.onPhaseFailure(RankFeaturePhase.this.getName(), "Computing updated ranks for results failed", e); } } ); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 209cadb3a0041..34622a5809d3f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -811,14 +811,15 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. */ + final String currentPhaseName = currentPhase.getName(); ShardOperationFailedException[] shardSearchFailures = AbstractSearchAsyncAction.buildShardFailures(shardFailures); if (shardSearchFailures.length == results.getNumShards()) { shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = shardSearchFailures.length == 0 ? null : ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> "All shards failed for phase: [" + currentPhase.getName() + "]", cause); - onPhaseFailure(currentPhase, "all shards failed", cause); + logger.debug(() -> "All shards failed for phase: [" + currentPhaseName + "]", cause); + onPhaseFailure(currentPhaseName, "all shards failed", cause); } else { Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; @@ -830,9 +831,9 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex int numShardFailures = shardSearchFailures.length; shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase.getName()), cause); + logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhaseName), cause); } - onPhaseFailure(currentPhase, "Partial shards failure", null); + onPhaseFailure(currentPhaseName, "Partial shards failure", null); } else { int discrepancy = results.getNumShards() - successfulOps.get(); assert discrepancy > 0 : "discrepancy: " + discrepancy; @@ -843,10 +844,10 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex successfulOps.get(), skippedOps.get(), results.getNumShards(), - currentPhase.getName() + currentPhaseName ); } - onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); + onPhaseFailure(currentPhaseName, "Partial shards failure (" + discrepancy + " shards unavailable)", null); } return; } @@ -857,7 +858,7 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex .collect(Collectors.joining(",")); logger.trace( "[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})", - currentPhase.getName(), + currentPhaseName, nextPhase.getName(), resultsFrom, clusterStateVersion @@ -874,7 +875,7 @@ private void executePhase(SearchPhase phase) { if (logger.isDebugEnabled()) { logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, phase.getName()), e); } - onPhaseFailure(phase, "", e); + onPhaseFailure(phase.getName(), "", e); } } @@ -885,9 +886,10 @@ private void executePhase(SearchPhase phase) { * @param msg an optional message * @param cause the cause of the phase failure */ - public void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { + @Override + public void onPhaseFailure(String phase, String msg, Throwable cause) { raisePhaseFailure( - new SearchPhaseExecutionException(phase.getName(), msg, cause, AbstractSearchAsyncAction.buildShardFailures(shardFailures)) + new SearchPhaseExecutionException(phase, msg, cause, AbstractSearchAsyncAction.buildShardFailures(shardFailures)) ); } diff --git a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java index 71bf2a47cfa47..47ff4ca6f0600 100644 --- a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java @@ -207,12 +207,7 @@ public void testOnPhaseFailure() { List> nodeLookups = new ArrayList<>(); ArraySearchPhaseResults phaseResults = phaseResults(requestIds, nodeLookups, 0); AbstractSearchAsyncAction action = createAction(searchRequest, phaseResults, listener, false, new AtomicLong()); - action.onPhaseFailure(new SearchPhase("test") { - @Override - public void run() { - - } - }, "message", null); + action.onPhaseFailure("test", "message", null); assertThat(exception.get(), instanceOf(SearchPhaseExecutionException.class)); SearchPhaseExecutionException searchPhaseExecutionException = (SearchPhaseExecutionException) exception.get(); assertEquals("message", searchPhaseExecutionException.getMessage()); diff --git a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java index cf65d756811ad..6c7ac7c48630a 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java +++ b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java @@ -113,7 +113,7 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At } @Override - public void onPhaseFailure(SearchPhase phase, String msg, Throwable cause) { + public void onPhaseFailure(String phase, String msg, Throwable cause) { phaseFailure.set(cause); } @@ -140,7 +140,7 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex try { nextPhase.run(); } catch (Exception e) { - onPhaseFailure(nextPhase, "phase failed", e); + onPhaseFailure(nextPhase.getName(), "phase failed", e); } } From b9c7e2549c13b523e79d5ebf6a14bd5eb95b539f Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 20 Dec 2024 21:14:07 +0100 Subject: [PATCH 047/132] cleaner --- .../action/search/AbstractSearchAsyncAction.java | 14 +------------- .../action/search/AsyncSearchContext.java | 15 +++++++++++++++ .../search/SearchQueryThenFetchAsyncAction.java | 9 ++++----- 3 files changed, 20 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index fca6f026f68ea..cf90e31bb1d8d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -57,6 +57,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.elasticsearch.action.search.AsyncSearchContext.buildShardFailures; import static org.elasticsearch.core.Strings.format; /** @@ -418,19 +419,6 @@ private void executePhase(SearchPhase phase) { } } - static ShardSearchFailure[] buildShardFailures(SetOnce> shardFailuresRef) { - AtomicArray shardFailures = shardFailuresRef.get(); - if (shardFailures == null) { - return ShardSearchFailure.EMPTY_ARRAY; - } - List entries = shardFailures.asList(); - ShardSearchFailure[] failures = new ShardSearchFailure[entries.size()]; - for (int i = 0; i < failures.length; i++) { - failures[i] = entries.get(i); - } - return failures; - } - private void onShardFailure(final int shardIndex, SearchShardTarget shard, final SearchShardIterator shardIt, Exception e) { // we always add the shard failure for a specific shard instance // we do make sure to clean it on a successful response from a shard diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 4f104cbf66402..873b90c62cf98 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -9,6 +9,7 @@ package org.elasticsearch.action.search; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; @@ -17,10 +18,24 @@ import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.transport.Transport; +import java.util.List; import java.util.function.Supplier; public interface AsyncSearchContext { + static ShardSearchFailure[] buildShardFailures(SetOnce> shardFailuresRef) { + AtomicArray shardFailures = shardFailuresRef.get(); + if (shardFailures == null) { + return ShardSearchFailure.EMPTY_ARRAY; + } + List entries = shardFailures.asList(); + ShardSearchFailure[] failures = new ShardSearchFailure[entries.size()]; + for (int i = 0; i < failures.length; i++) { + failures[i] = entries.get(i); + } + return failures; + } + SearchRequest getRequest(); void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 34622a5809d3f..a71247b82102a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -82,6 +82,7 @@ import java.util.stream.Collectors; import static org.elasticsearch.action.search.AbstractSearchAsyncAction.DEFAULT_INDEX_BOOST; +import static org.elasticsearch.action.search.AsyncSearchContext.buildShardFailures; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; import static org.elasticsearch.core.Strings.format; @@ -227,7 +228,7 @@ public SearchRequest getRequest() { * @param queryResults the results of the query phase */ public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { - ShardSearchFailure[] failures = AbstractSearchAsyncAction.buildShardFailures(shardFailures); + ShardSearchFailure[] failures = buildShardFailures(shardFailures); Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; if (allowPartialResults == false && failures.length > 0) { @@ -812,7 +813,7 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex * fail. Otherwise we continue to the next phase. */ final String currentPhaseName = currentPhase.getName(); - ShardOperationFailedException[] shardSearchFailures = AbstractSearchAsyncAction.buildShardFailures(shardFailures); + ShardOperationFailedException[] shardSearchFailures = buildShardFailures(shardFailures); if (shardSearchFailures.length == results.getNumShards()) { shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = shardSearchFailures.length == 0 @@ -888,9 +889,7 @@ private void executePhase(SearchPhase phase) { */ @Override public void onPhaseFailure(String phase, String msg, Throwable cause) { - raisePhaseFailure( - new SearchPhaseExecutionException(phase, msg, cause, AbstractSearchAsyncAction.buildShardFailures(shardFailures)) - ); + raisePhaseFailure(new SearchPhaseExecutionException(phase, msg, cause, buildShardFailures(shardFailures))); } @Override From 6fbd204ca79ce9617039960e2ec36ef1e8b57215 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 20 Dec 2024 21:23:21 +0100 Subject: [PATCH 048/132] cleanup --- .../action/search/AbstractSearchAsyncAction.java | 7 ------- .../org/elasticsearch/action/search/DfsQueryPhase.java | 7 ++++++- .../action/search/FetchLookupFieldsPhase.java | 8 ++++++-- .../elasticsearch/action/search/FetchSearchPhase.java | 8 ++++++-- .../elasticsearch/action/search/RankFeaturePhase.java | 10 +++++++--- .../action/search/SearchQueryThenFetchAsyncAction.java | 7 ++----- 6 files changed, 27 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index cf90e31bb1d8d..cbb0227a537a6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -577,13 +577,6 @@ public final int getNumShards() { return results.getNumShards(); } - /** - * Returns a logger for this context to prevent each individual phase to create their own logger. - */ - public final Logger getLogger() { - return logger; - } - /** * Returns the currently executing search task */ diff --git a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java index 7f1a29677b04e..41911e649ef3c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java @@ -8,6 +8,8 @@ */ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.join.ScoreMode; import org.elasticsearch.common.lucene.Lucene; @@ -39,6 +41,9 @@ * @see CountedCollector#onFailure(int, SearchShardTarget, Exception) */ final class DfsQueryPhase extends SearchPhase { + + private static final Logger logger = LogManager.getLogger(DfsQueryPhase.class); + private final SearchPhaseResults queryResult; private final List searchResults; private final AggregatedDfs dfs; @@ -135,7 +140,7 @@ private void shardFailure( SearchShardTarget shardTarget, CountedCollector counter ) { - context.getLogger().debug(() -> "[" + querySearchRequest.contextId() + "] Failed to execute query phase", exception); + logger.debug(() -> "[" + querySearchRequest.contextId() + "] Failed to execute query phase", exception); progressListener.notifyQueryFailure(shardIndex, shardTarget, exception); counter.onFailure(shardIndex, shardTarget, exception); } diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java index d09a1b16dfd63..e40b3742daecc 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java @@ -125,7 +125,7 @@ public void onResponse(MultiSearchResponse items) { } } if (failure != null) { - context.onPhaseFailure(FetchLookupFieldsPhase.this.getName(), "failed to fetch lookup fields", failure); + failPhase(failure); } else { context.sendSearchResponse(searchResponse, queryResults); } @@ -133,8 +133,12 @@ public void onResponse(MultiSearchResponse items) { @Override public void onFailure(Exception e) { - context.onPhaseFailure(FetchLookupFieldsPhase.this.getName(), "failed to fetch lookup fields", e); + failPhase(e); } }); } + + private void failPhase(Exception e) { + context.onPhaseFailure(FetchLookupFieldsPhase.this.getName(), "failed to fetch lookup fields", e); + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 3c9d70fc9144a..0b408acd08ac1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -95,11 +95,15 @@ protected void doRun() throws Exception { @Override public void onFailure(Exception e) { - context.onPhaseFailure(FetchSearchPhase.this.getName(), "", e); + failPhase(e); } }); } + private void failPhase(Exception e) { + context.onPhaseFailure(FetchSearchPhase.this.getName(), "", e); + } + private void innerRun() throws Exception { assert this.reducedQueryPhase == null ^ this.resultConsumer == null; // depending on whether we executed the RankFeaturePhase we may or may not have the reduced query result computed already @@ -216,7 +220,7 @@ public void innerOnResponse(FetchSearchResult result) { progressListener.notifyFetchResult(shardIndex); counter.onResult(result); } catch (Exception e) { - context.onPhaseFailure(FetchSearchPhase.this.getName(), "", e); + failPhase(e); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java index 3ce3fce7ab376..2456f2512692c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java @@ -76,11 +76,15 @@ protected void doRun() throws Exception { @Override public void onFailure(Exception e) { - context.onPhaseFailure(RankFeaturePhase.this.getName(), "", e); + failPhase("", e); } }); } + private void failPhase(String msg, Exception e) { + context.onPhaseFailure(RankFeaturePhase.this.getName(), msg, e); + } + void innerRun(RankFeaturePhaseRankCoordinatorContext rankFeaturePhaseRankCoordinatorContext) throws Exception { // if the RankBuilder specifies a QueryPhaseCoordinatorContext, it will be called as part of the reduce call // to operate on the first `rank_window_size * num_shards` results and merge them appropriately. @@ -131,7 +135,7 @@ protected void innerOnResponse(RankFeatureResult response) { progressListener.notifyRankFeatureResult(shardIndex); rankRequestCounter.onResult(response); } catch (Exception e) { - context.onPhaseFailure(RankFeaturePhase.this.getName(), "", e); + failPhase("", e); } } @@ -186,7 +190,7 @@ public void onResponse(RankFeatureDoc[] docsWithUpdatedScores) { @Override public void onFailure(Exception e) { - context.onPhaseFailure(RankFeaturePhase.this.getName(), "Computing updated ranks for results failed", e); + failPhase("Computing updated ranks for results failed", e); } } ); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index a71247b82102a..ad12a3458be9f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -101,7 +101,6 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn private final BiFunction nodeIdToConnection; private final SearchTask task; protected final SearchPhaseResults results; - private final long clusterStateVersion; private final TransportVersion minTransportVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; @@ -175,7 +174,6 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); this.nodeIdToConnection = nodeIdToConnection; this.concreteIndexBoosts = concreteIndexBoosts; - this.clusterStateVersion = clusterState.version(); this.minTransportVersion = clusterState.getMinTransportVersion(); this.aliasFilter = aliasFilter; this.results = resultConsumer; @@ -858,11 +856,10 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex .map(r -> r.getSearchShardTarget().toString()) .collect(Collectors.joining(",")); logger.trace( - "[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})", + "[{}] Moving to next phase: [{}], based on results from: {}", currentPhaseName, nextPhase.getName(), - resultsFrom, - clusterStateVersion + resultsFrom ); } executePhase(nextPhase); From d2dae7b4c52f63c0a7be1d4e2b7e4b942b1f564b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 20 Dec 2024 21:45:21 +0100 Subject: [PATCH 049/132] cleanup --- .../search/AbstractSearchAsyncAction.java | 49 +++++++++---------- .../action/search/AsyncSearchContext.java | 2 +- .../action/search/DfsQueryPhase.java | 2 +- .../action/search/ExpandSearchPhase.java | 2 +- .../action/search/FetchSearchPhase.java | 2 +- .../action/search/RankFeaturePhase.java | 2 +- .../SearchQueryThenFetchAsyncAction.java | 33 ++++++------- .../action/search/MockSearchPhaseContext.java | 2 +- 8 files changed, 43 insertions(+), 51 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index cbb0227a537a6..ab786cb7d819d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -206,7 +206,7 @@ long buildTookInMillis() { * This is the main entry point for a search. This method starts the search execution of the initial phase. */ public final void start() { - if (getNumShards() == 0) { + if (results.getNumShards() == 0) { // no search shards to search on, bail with empty response // (it happens with search across _all with no indices around and consistent with broadcast operations) int trackTotalHitsUpTo = request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO @@ -346,24 +346,24 @@ protected abstract void executePhaseOnShard( * of the next phase. If there are no successful operations in the context when this method is executed the search is aborted and * a response is returned to the user indicating that all shards have failed. */ - public void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { + public void executeNextPhase(String currentPhase, Supplier nextPhaseSupplier) { /* This is the main search phase transition where we move to the next phase. If all shards * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. */ - final String currentPhaseName = currentPhase.getName(); ShardOperationFailedException[] shardSearchFailures = buildShardFailures(shardFailures); - if (shardSearchFailures.length == getNumShards()) { + final int numShards = results.getNumShards(); + if (shardSearchFailures.length == numShards) { shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = shardSearchFailures.length == 0 ? null : ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> "All shards failed for phase: [" + currentPhaseName + "]", cause); - onPhaseFailure(currentPhaseName, "all shards failed", cause); + logger.debug(() -> "All shards failed for phase: [" + currentPhase + "]", cause); + onPhaseFailure(currentPhase, "all shards failed", cause); } else { Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; - if (allowPartialResults == false && successfulOps.get() != getNumShards()) { + if (allowPartialResults == false && successfulOps.get() != numShards) { // check if there are actual failures in the atomic array since // successful retries can reset the failures to null if (shardSearchFailures.length > 0) { @@ -371,11 +371,11 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex int numShardFailures = shardSearchFailures.length; shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhaseName), cause); + logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase), cause); } - onPhaseFailure(currentPhaseName, "Partial shards failure", null); + onPhaseFailure(currentPhase, "Partial shards failure", null); } else { - int discrepancy = getNumShards() - successfulOps.get(); + int discrepancy = numShards - successfulOps.get(); assert discrepancy > 0 : "discrepancy: " + discrepancy; if (logger.isDebugEnabled()) { logger.debug( @@ -383,11 +383,11 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex discrepancy, successfulOps.get(), skippedOps.get(), - getNumShards(), - currentPhaseName + numShards, + currentPhase ); } - onPhaseFailure(currentPhaseName, "Partial shards failure (" + discrepancy + " shards unavailable)", null); + onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); } return; } @@ -398,7 +398,7 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex .collect(Collectors.joining(",")); logger.trace( "[{}] Moving to next phase: [{}], based on results from: {} (cluster state version: {})", - currentPhaseName, + currentPhase, nextPhase.getName(), resultsFrom, clusterStateVersion @@ -489,7 +489,7 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, synchronized (shardFailuresMutex) { shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it? if (shardFailures == null) { // still null so we are the first and create a new instance - shardFailures = new AtomicArray<>(getNumShards()); + shardFailures = new AtomicArray<>(results.getNumShards()); this.shardFailures.set(shardFailures); } } @@ -570,13 +570,6 @@ private void successfulShardExecution(SearchShardIterator shardsIt) { } } - /** - * Returns the total number of shards to the current search across all indices - */ - public final int getNumShards() { - return results.getNumShards(); - } - /** * Returns the currently executing search task */ @@ -627,12 +620,13 @@ private SearchResponse buildSearchResponse( ) { int numSuccess = successfulOps.get(); int numFailures = failures.length; - assert numSuccess + numFailures == getNumShards() - : "numSuccess(" + numSuccess + ") + numFailures(" + numFailures + ") != totalShards(" + getNumShards() + ")"; + final int numShards = results.getNumShards(); + assert numSuccess + numFailures == numShards + : "numSuccess(" + numSuccess + ") + numFailures(" + numFailures + ") != totalShards(" + numShards + ")"; return new SearchResponse( internalSearchResponse, scrollId, - getNumShards(), + numShards, numSuccess, skippedOps.get(), buildTookInMillis(), @@ -731,13 +725,14 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C * @see #onShardResult(SearchPhaseResult, SearchShardIterator) */ private void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() - executeNextPhase(this, this::getNextPhase); + executeNextPhase(this.getName(), this::getNextPhase); } /** * Returns a connection to the node if connected otherwise and {@link org.elasticsearch.transport.ConnectTransportException} will be * thrown. */ + @Override public final Transport.Connection getConnection(String clusterAlias, String nodeId) { return nodeIdToConnection.apply(clusterAlias, nodeId); } @@ -769,7 +764,7 @@ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator s request, shardIt.shardId(), shardIndex, - getNumShards(), + results.getNumShards(), filter, indexBoost, timeProvider.absoluteStartMillis(), diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 873b90c62cf98..9a56ed39f6a8c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -58,5 +58,5 @@ static ShardSearchFailure[] buildShardFailures(SetOnce nextPhaseSupplier); + void executeNextPhase(String currentPhase, Supplier nextPhaseSupplier); } diff --git a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java index 41911e649ef3c..872dfc56e0516 100644 --- a/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/DfsQueryPhase.java @@ -79,7 +79,7 @@ public void run() { final CountedCollector counter = new CountedCollector<>( queryResult, searchResults.size(), - () -> context.executeNextPhase(this, () -> nextPhaseFactory.apply(queryResult)), + () -> context.executeNextPhase(this.getName(), () -> nextPhaseFactory.apply(queryResult)), context ); diff --git a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java index c2529be244dad..a2b2a5c233083 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java @@ -168,6 +168,6 @@ private static SearchSourceBuilder buildExpandSearchSourceBuilder(InnerHitBuilde } private void onPhaseDone() { - context.executeNextPhase(this, nextPhase); + context.executeNextPhase(getName(), nextPhase); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 0b408acd08ac1..5f9b4934aea85 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -267,7 +267,7 @@ private void moveToNextPhase( AtomicArray fetchResultsArr, SearchPhaseController.ReducedQueryPhase reducedQueryPhase ) { - context.executeNextPhase(this, () -> { + context.executeNextPhase(this.getName(), () -> { var resp = SearchPhaseController.merge(context.getRequest().scroll() != null, reducedQueryPhase, fetchResultsArr); context.addReleasable(resp); return nextPhaseFactory.apply(resp, searchPhaseShardResults); diff --git a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java index 2456f2512692c..c9d5db09b8ad2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java @@ -235,6 +235,6 @@ private float maxScore(ScoreDoc[] scoreDocs) { } void moveToNextPhase(SearchPhaseResults phaseResults, SearchPhaseController.ReducedQueryPhase reducedQueryPhase) { - context.executeNextPhase(this, () -> new FetchSearchPhase(phaseResults, aggregatedDfs, context, reducedQueryPhase)); + context.executeNextPhase(this.getName(), () -> new FetchSearchPhase(phaseResults, aggregatedDfs, context, reducedQueryPhase)); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index ad12a3458be9f..9dcb52621abb6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -743,7 +743,7 @@ private void finishShardAndMaybePhase(ShardId shardId) { boolean removed = outstandingShards.remove(shardId); assert removed : "unknown shardId " + shardId; if (outstandingShards.isEmpty()) { - executeNextPhase(this, this::getNextPhase); + executeNextPhase(this.getName(), this::getNextPhase); } } @@ -805,24 +805,24 @@ private void successfulShardExecution(SearchShardIterator shardsIt) { } @Override - public void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { + public void executeNextPhase(String currentPhase, Supplier nextPhaseSupplier) { /* This is the main search phase transition where we move to the next phase. If all shards * failed or if there was a failure and partial results are not allowed, then we immediately * fail. Otherwise we continue to the next phase. */ - final String currentPhaseName = currentPhase.getName(); ShardOperationFailedException[] shardSearchFailures = buildShardFailures(shardFailures); - if (shardSearchFailures.length == results.getNumShards()) { + final int numShards = results.getNumShards(); + if (shardSearchFailures.length == numShards) { shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = shardSearchFailures.length == 0 ? null : ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> "All shards failed for phase: [" + currentPhaseName + "]", cause); - onPhaseFailure(currentPhaseName, "all shards failed", cause); + logger.debug(() -> "All shards failed for phase: [" + currentPhase + "]", cause); + onPhaseFailure(currentPhase, "all shards failed", cause); } else { Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; - if (allowPartialResults == false && successfulOps.get() != results.getNumShards()) { + if (allowPartialResults == false && successfulOps.get() != numShards) { // check if there are actual failures in the atomic array since // successful retries can reset the failures to null if (shardSearchFailures.length > 0) { @@ -830,11 +830,11 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex int numShardFailures = shardSearchFailures.length; shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhaseName), cause); + logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase), cause); } - onPhaseFailure(currentPhaseName, "Partial shards failure", null); + onPhaseFailure(currentPhase, "Partial shards failure", null); } else { - int discrepancy = results.getNumShards() - successfulOps.get(); + int discrepancy = numShards - successfulOps.get(); assert discrepancy > 0 : "discrepancy: " + discrepancy; if (logger.isDebugEnabled()) { logger.debug( @@ -842,24 +842,21 @@ public void executeNextPhase(SearchPhase currentPhase, Supplier nex discrepancy, successfulOps.get(), skippedOps.get(), - results.getNumShards(), - currentPhaseName + numShards, + currentPhase ); } - onPhaseFailure(currentPhaseName, "Partial shards failure (" + discrepancy + " shards unavailable)", null); + onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); } return; } var nextPhase = nextPhaseSupplier.get(); if (logger.isTraceEnabled()) { - final String resultsFrom = results.getSuccessfulResults() - .map(r -> r.getSearchShardTarget().toString()) - .collect(Collectors.joining(",")); logger.trace( "[{}] Moving to next phase: [{}], based on results from: {}", - currentPhaseName, + currentPhase, nextPhase.getName(), - resultsFrom + results.getSuccessfulResults().map(r -> r.getSearchShardTarget().toString()).collect(Collectors.joining(",")) ); } executePhase(nextPhase); diff --git a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java index 6c7ac7c48630a..e8e12300c23e3 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java +++ b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java @@ -135,7 +135,7 @@ public SearchTransportService getSearchTransport() { } @Override - public void executeNextPhase(SearchPhase currentPhase, Supplier nextPhaseSupplier) { + public void executeNextPhase(String currentPhase, Supplier nextPhaseSupplier) { var nextPhase = nextPhaseSupplier.get(); try { nextPhase.run(); From 29e610a62552a1f300dfce5397a39bbbf663c087 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 30 Dec 2024 15:02:26 +0100 Subject: [PATCH 050/132] debug --- .../search/ArraySearchPhaseResults.java | 2 +- .../search/QueryPhaseResultConsumer.java | 14 ++++----- .../SearchQueryThenFetchAsyncAction.java | 29 ++++++++++--------- 3 files changed, 24 insertions(+), 21 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java b/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java index e4d1f9f63bd2b..516d6c6b3eff1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java +++ b/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java @@ -43,7 +43,7 @@ Stream getSuccessfulResults() { @Override void consumeResult(Result result, Runnable next) { assert results.get(result.getShardIndex()) == null : "shardIndex: " + result.getShardIndex() + " is already set"; - results.set(result.getShardIndex(), result); + results.setOnce(result.getShardIndex(), result); result.incRef(); next.run(); } diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index d5c57520151b8..441e1f17dcb89 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -161,6 +161,7 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { super.consumeResult(result, () -> {}); QuerySearchResult querySearchResult = result.queryResult(); progressListener.notifyQueryResult(querySearchResult.getShardIndex(), querySearchResult); + assert result.getShardIndex() == querySearchResult.getShardIndex(); consume(querySearchResult, next); } @@ -179,8 +180,13 @@ public void reduce(Object[] results, TopDocsStats topDocsStats, MergeResult merg } } + private final AtomicReference reduced = new AtomicReference<>(null); + @Override public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { + if (reduced.compareAndSet(null, new RuntimeException()) == false) { + throw new AssertionError(); + } if (hasPendingMerges()) { throw new AssertionError("partial reduce in-flight"); } @@ -190,7 +196,7 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { } // ensure consistent ordering - sortBuffer(); + buffer.sort(RESULT_COMPARATOR); final TopDocsStats topDocsStats = this.topDocsStats; final int resultSize = buffer.size() + (mergeResult == null ? 0 : 1); final List topDocsList = hasTopDocs ? new ArrayList<>(resultSize) : null; @@ -345,12 +351,6 @@ private boolean hasPendingMerges() { return queue.isEmpty() == false || runningTask.get() != null; } - void sortBuffer() { - if (buffer.size() > 0) { - buffer.sort(RESULT_COMPARATOR); - } - } - private synchronized void addWithoutBreaking(long size) { circuitBreaker.addWithoutBreaking(size); circuitBreakerBytes += size; diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 9dcb52621abb6..b7ee62efe3285 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -77,6 +77,7 @@ import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -432,7 +433,7 @@ protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget progressListener.notifyQueryFailure(shardIndex, shardTarget, exc); } - protected void onShardResult(SearchPhaseResult result, SearchShardIterator shardIt) { + protected void onShardResult(SearchPhaseResult result) { QuerySearchResult queryResult = result.queryResult(); if (queryResult.isNull() == false // disable sort optims for scroll requests because they keep track of the last bottom doc locally (per shard) @@ -457,7 +458,7 @@ protected void onShardResult(SearchPhaseResult result, SearchShardIterator shard if (logger.isTraceEnabled()) { logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); } - results.consumeResult(result, () -> onShardResultConsumed(result, shardIt)); + results.consumeResult(result, () -> onShardResultConsumed(result)); } static SearchPhase nextPhase( @@ -540,7 +541,7 @@ public void run() throws IOException { getOriginalIndices(shardIndex), shardIndex, routing.getShardId(), - shardIterators[shardIndex].getSearchContextId() + shardRoutings.getSearchContextId() ) ); } else { @@ -616,7 +617,7 @@ public void handleResponse(NodeQueryResponse response) { @Override public void handleException(TransportException e) { for (ShardToQuery shard : request.shards) { - var shardIt = shardsIts.get(shard.shardIndex); + var shardIt = shardIterators[shard.shardIndex]; onShardFailure( shard.shardIndex, new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), @@ -690,7 +691,7 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat @Override public void innerOnResponse(SearchPhaseResult result) { try { - onShardResult(result, shardIt); + onShardResult(result); } catch (Exception exc) { onShardFailure(shardIndex, shard, shardIt, exc); } @@ -739,10 +740,15 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } } + private final AtomicReference done = new AtomicReference<>(null); + private void finishShardAndMaybePhase(ShardId shardId) { boolean removed = outstandingShards.remove(shardId); assert removed : "unknown shardId " + shardId; if (outstandingShards.isEmpty()) { + if (done.compareAndSet(null, new RuntimeException("successful ops " + successfulOps.get())) == false) { + throw new AssertionError("failed to finish shard", done.get()); + } executeNextPhase(this.getName(), this::getNextPhase); } } @@ -788,6 +794,7 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, if (results.hasResult(shardIndex)) { assert failure == null : "shard failed before but shouldn't: " + failure; + outstandingShards.add(shardTarget.getShardId()); successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter } } @@ -797,11 +804,7 @@ void skipShard(SearchShardIterator iterator) { successfulOps.incrementAndGet(); skippedOps.incrementAndGet(); assert iterator.skip(); - successfulShardExecution(iterator); - } - - private void successfulShardExecution(SearchShardIterator shardsIt) { - finishShardAndMaybePhase(shardsIt.shardId()); + finishShardAndMaybePhase(iterator.shardId()); } @Override @@ -931,7 +934,7 @@ public boolean isPartOfPointInTime(ShardSearchContextId contextId) { return AbstractSearchAsyncAction.isPartOfPIT(namedWriteableRegistry, request, contextId); } - private void onShardResultConsumed(SearchPhaseResult result, SearchShardIterator shardIt) { + private void onShardResultConsumed(SearchPhaseResult result) { successfulOps.incrementAndGet(); // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level // so its ok concurrency wise to miss potentially the shard failures being created because of another failure @@ -945,7 +948,7 @@ private void onShardResultConsumed(SearchPhaseResult result, SearchShardIterator // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc. // increment all the "future" shards to update the total ops since we some may work and some may not... // and when that happens, we break on total ops, so we must maintain them - successfulShardExecution(shardIt); + finishShardAndMaybePhase(result.getSearchShardTarget().getShardId()); } public static final String NODE_SEARCH_ACTION_NAME = "indices:data/read/search[query][n]"; @@ -976,7 +979,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, request.shards.size(), - e -> {} + e -> logger.error("failed to merge on data node", e) ); final Runnable onDone = () -> { if (countDown.countDown()) { From 685833f41c08433993e08206500c716d3a2c807b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 2 Jan 2025 16:28:41 +0100 Subject: [PATCH 051/132] facepalm --- .../SearchQueryThenFetchAsyncAction.java | 52 ++++++++++++------- 1 file changed, 32 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index b7ee62efe3285..b9cba3550ce22 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -580,7 +580,6 @@ public Executor executor() { @Override public void handleResponse(NodeQueryResponse response) { - int failedShards = 0; for (int i = 0; i < response.results.length; i++) { var s = request.shards.get(i); int shardIdx = s.shardIndex; @@ -591,7 +590,6 @@ public void handleResponse(NodeQueryResponse response) { shardIterators[shardIdx], e ); - failedShards++; } else if (response.results[i] instanceof QuerySearchResult q) { q.setShardIndex(shardIdx); var shardId = s.shardId(); @@ -600,16 +598,20 @@ public void handleResponse(NodeQueryResponse response) { ); } } - final int successfulShards = request.shards.size() - failedShards; - if (successfulShards > 0) { - hasShardResponse.set(true); - for (Object result : response.results) { - if (result instanceof SearchPhaseResult searchPhaseResult) { - results.consumeResult(searchPhaseResult, () -> { - successfulOps.incrementAndGet(); - finishShardAndMaybePhase(searchPhaseResult.getSearchShardTarget().getShardId()); - }); - } + for (Object result : response.results) { + if (result instanceof SearchPhaseResult searchPhaseResult) { + hasShardResponse.set(true); + results.consumeResult(searchPhaseResult, () -> { + successfulOps.incrementAndGet(); + var shard = searchPhaseResult.getSearchShardTarget().getShardId(); + logger.info( + "--> bulk result for shard [{}][{}][{}]", + shard, + searchPhaseResult.getShardIndex(), + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) + ); + finishShardAndMaybePhase(shard); + }); } } } @@ -691,8 +693,21 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat @Override public void innerOnResponse(SearchPhaseResult result) { try { + logger.info( + "--> result for shard [{}][{}][{}]", + shard.getShardId(), + shardIndex, + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) + ); onShardResult(result); } catch (Exception exc) { + logger.error( + "--> failure for shard [{}][{}][{}]", + shard.getShardId(), + shardIndex, + System.identityHashCode(SearchQueryThenFetchAsyncAction.this), + exc + ); onShardFailure(shardIndex, shard, shardIt, exc); } } @@ -731,12 +746,9 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } } onShardGroupFailure(shardIndex, shard, e); - } - if (lastShard == false) { - performPhaseOnShard(shardIndex, shardIt, nextShard); + finishShardAndMaybePhase(shard.getShardId()); } else { - final ShardId shardId = shardIt.shardId(); - finishShardAndMaybePhase(shardId); + performPhaseOnShard(shardIndex, shardIt, nextShard); } } @@ -746,10 +758,10 @@ private void finishShardAndMaybePhase(ShardId shardId) { boolean removed = outstandingShards.remove(shardId); assert removed : "unknown shardId " + shardId; if (outstandingShards.isEmpty()) { - if (done.compareAndSet(null, new RuntimeException("successful ops " + successfulOps.get())) == false) { - throw new AssertionError("failed to finish shard", done.get()); + var doneTrace = new RuntimeException("successful ops " + successfulOps.get()); + if (done.compareAndSet(null, doneTrace)) { + executeNextPhase(this.getName(), this::getNextPhase); } - executeNextPhase(this.getName(), this::getNextPhase); } } From 1d7554fe47b88d54f610466f94260fc7e9e328cd Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 2 Jan 2025 17:51:42 +0100 Subject: [PATCH 052/132] facepalm --- .../search/SearchQueryThenFetchAsyncAction.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index b9cba3550ce22..76e8103dd909f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -702,10 +702,13 @@ public void innerOnResponse(SearchPhaseResult result) { onShardResult(result); } catch (Exception exc) { logger.error( - "--> failure for shard [{}][{}][{}]", - shard.getShardId(), - shardIndex, - System.identityHashCode(SearchQueryThenFetchAsyncAction.this), + "--> failure for shard [" + + shard.getShardId() + + "][" + + shardIndex + + "][" + + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) + + "]", exc ); onShardFailure(shardIndex, shard, shardIt, exc); From c8d83295aefd4d684f9b58d31c91edfbb4f9d954 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 2 Jan 2025 18:41:11 +0100 Subject: [PATCH 053/132] security --- .../search/SearchQueryThenFetchAsyncAction.java | 14 +++++++++++++- .../security/authz/PreAuthorizationUtils.java | 4 +++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 76e8103dd909f..710e9a9409aa5 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -17,10 +17,12 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.support.ChannelActionListener; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; @@ -360,7 +362,7 @@ public void writeTo(StreamOutput out) throws IOException { } } - public static class NodeQueryRequest extends TransportRequest { + public static class NodeQueryRequest extends TransportRequest implements IndicesRequest { private final List shards; private final SearchRequest searchRequest; private final Map aliasFilters; @@ -399,6 +401,16 @@ public void writeTo(StreamOutput out) throws IOException { out.writeMap(aliasFilters, (o, v) -> v.writeTo(o)); out.writeVInt(totalShards); } + + @Override + public String[] indices() { + return shards.stream().map(s -> s.originalIndices().indices()).flatMap(Arrays::stream).distinct().toArray(String[]::new); + } + + @Override + public IndicesOptions indicesOptions() { + return shards.getFirst().originalIndices.indicesOptions(); + } } private record ShardToQuery( diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java index 221b7a65e1f8f..3c8f5dd5cdff9 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authz/PreAuthorizationUtils.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction; import org.elasticsearch.action.search.SearchTransportService; import org.elasticsearch.action.search.TransportSearchAction; import org.elasticsearch.xpack.core.security.SecurityContext; @@ -46,7 +47,8 @@ public final class PreAuthorizationUtils { SearchTransportService.QUERY_ID_ACTION_NAME, SearchTransportService.FETCH_ID_ACTION_NAME, SearchTransportService.RANK_FEATURE_SHARD_ACTION_NAME, - SearchTransportService.QUERY_CAN_MATCH_NODE_NAME + SearchTransportService.QUERY_CAN_MATCH_NODE_NAME, + SearchQueryThenFetchAsyncAction.NODE_SEARCH_ACTION_NAME ) ); From e41241753d4325b7080b571a8da15530220921ef Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 2 Jan 2025 23:07:53 +0100 Subject: [PATCH 054/132] fix --- .../elasticsearch/action/search/SearchShardIterator.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java b/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java index bc5f56f572e9f..b543ff922886f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchShardIterator.java @@ -103,14 +103,12 @@ public String getClusterAlias() { return clusterAlias; } - public SearchShardTarget current; - SearchShardTarget nextOrNull() { final String nodeId = targetNodesIterator.nextOrNull(); if (nodeId != null) { - return current = new SearchShardTarget(nodeId, shardId, clusterAlias); + return new SearchShardTarget(nodeId, shardId, clusterAlias); } - return current = null; + return null; } int remaining() { From 08ebe4dfb55eaeb76ab0a183981bcbfd47199a34 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 3 Jan 2025 02:49:39 +0100 Subject: [PATCH 055/132] fix --- .../SearchQueryThenFetchAsyncAction.java | 96 ++++++++++--------- 1 file changed, 49 insertions(+), 47 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 710e9a9409aa5..121f4cfc44aa5 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -73,10 +73,8 @@ import java.util.Map; import java.util.Objects; import java.util.Set; -import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.Executor; -import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -120,7 +118,7 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn private final SearchShardIterator[] shardIterators; private final AtomicBoolean requestCancelled = new AtomicBoolean(); - private final Set outstandingShards = ConcurrentCollections.newConcurrentSet(); + private final Set outstandingShards = ConcurrentCollections.newConcurrentSet(); // protected for tests protected final List releasables = new ArrayList<>(); @@ -517,14 +515,11 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) @Override public void run() throws IOException { // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging - for (SearchShardIterator toSkipShardsIt : toSkipShardsIts) { - outstandingShards.add(toSkipShardsIt.shardId()); - } final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); for (int i = 0; i < shardIterators.length; i++) { var iterator = shardIterators[i]; shardIndexMap.put(iterator, i); - outstandingShards.add(iterator.shardId()); + outstandingShards.add(i); } for (final SearchShardIterator iterator : toSkipShardsIts) { assert iterator.skip(); @@ -602,11 +597,10 @@ public void handleResponse(NodeQueryResponse response) { shardIterators[shardIdx], e ); - } else if (response.results[i] instanceof QuerySearchResult q) { + } else if (response.results[i] instanceof SearchPhaseResult q) { q.setShardIndex(shardIdx); - var shardId = s.shardId(); q.setSearchShardTarget( - new SearchShardTarget(nodeId, shardId, request.searchRequest.getLocalClusterAlias()) + new SearchShardTarget(nodeId, s.shardId(), request.searchRequest.getLocalClusterAlias()) ); } } @@ -616,13 +610,14 @@ public void handleResponse(NodeQueryResponse response) { results.consumeResult(searchPhaseResult, () -> { successfulOps.incrementAndGet(); var shard = searchPhaseResult.getSearchShardTarget().getShardId(); + final int shardIndex = searchPhaseResult.getShardIndex(); logger.info( "--> bulk result for shard [{}][{}][{}]", shard, - searchPhaseResult.getShardIndex(), + shardIndex, System.identityHashCode(SearchQueryThenFetchAsyncAction.this) ); - finishShardAndMaybePhase(shard); + finishShardAndMaybePhase(shardIndex); }); } } @@ -652,6 +647,8 @@ public void handleException(TransportException e) { } } }); + } else { + finishIfAllDone(); } } @@ -761,7 +758,7 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } } onShardGroupFailure(shardIndex, shard, e); - finishShardAndMaybePhase(shard.getShardId()); + finishShardAndMaybePhase(shardIndex); } else { performPhaseOnShard(shardIndex, shardIt, nextShard); } @@ -769,9 +766,22 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final private final AtomicReference done = new AtomicReference<>(null); - private void finishShardAndMaybePhase(ShardId shardId) { - boolean removed = outstandingShards.remove(shardId); - assert removed : "unknown shardId " + shardId; + private void finishShardAndMaybePhase(int shardIndex) { + boolean removed = outstandingShards.remove(shardIndex); + var shardId = shardIterators[shardIndex].shardId(); + var shardIdString = "[" + + shardId + + "] [" + + shardId.getIndex().getUUID() + + "][" + + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) + + "]"; + logger.info("finishing shard [{}]", shardIdString); + assert removed : "unknown shardId " + shardIdString; + finishIfAllDone(); + } + + private void finishIfAllDone() { if (outstandingShards.isEmpty()) { var doneTrace = new RuntimeException("successful ops " + successfulOps.get()); if (done.compareAndSet(null, doneTrace)) { @@ -821,7 +831,7 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, if (results.hasResult(shardIndex)) { assert failure == null : "shard failed before but shouldn't: " + failure; - outstandingShards.add(shardTarget.getShardId()); + outstandingShards.add(shardIndex); successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter } } @@ -831,7 +841,6 @@ void skipShard(SearchShardIterator iterator) { successfulOps.incrementAndGet(); skippedOps.incrementAndGet(); assert iterator.skip(); - finishShardAndMaybePhase(iterator.shardId()); } @Override @@ -975,7 +984,7 @@ private void onShardResultConsumed(SearchPhaseResult result) { // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc. // increment all the "future" shards to update the total ops since we some may work and some may not... // and when that happens, we break on total ops, so we must maintain them - finishShardAndMaybePhase(result.getSearchShardTarget().getShardId()); + finishShardAndMaybePhase(result.getShardIndex()); } public static final String NODE_SEARCH_ACTION_NAME = "indices:data/read/search[query][n]"; @@ -987,14 +996,13 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo EsExecutors.DIRECT_EXECUTOR_SERVICE, NodeQueryRequest::new, (request, channel, task) -> { - final BlockingQueue shards = new LinkedBlockingQueue<>(request.shards); final int workers = Math.min( request.shards.size(), transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax() ); var executor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH); final ConcurrentHashMap failures = new ConcurrentHashMap<>(); - final AtomicInteger shardIndex = new AtomicInteger(); + // TODO: start at 0 request.searchRequest.finalReduce = false; request.searchRequest.setBatchedReduceSize(Integer.MAX_VALUE); final CountDown countDown = new CountDown(request.shards.size()); @@ -1060,24 +1068,21 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo } } }; + final AtomicInteger shardIndex = new AtomicInteger(workers - 1); for (int i = 0; i < workers; i++) { - ShardToQuery shardToQuery = shards.poll(); - if (shardToQuery != null) { - executor.execute( - shardTask( - searchService, - request, - (CancellableTask) task, - shardToQuery, - shardIndex, - shards, - executor, - queryPhaseResultConsumer, - failures, - onDone - ) - ); - } + executor.execute( + shardTask( + searchService, + request, + (CancellableTask) task, + i, + shardIndex, + executor, + queryPhaseResultConsumer, + failures, + onDone + ) + ); } } ); @@ -1097,16 +1102,15 @@ private static AbstractRunnable shardTask( SearchService searchService, NodeQueryRequest request, CancellableTask task, - ShardToQuery shardToQuery, + int dataNodeLocalIdx, AtomicInteger shardIndex, - BlockingQueue shards, Executor executor, QueryPhaseResultConsumer queryPhaseResultConsumer, Map failures, Runnable onDone ) { var pitBuilder = request.searchRequest.pointInTimeBuilder(); - final int dataNodeLocalIdx = shardIndex.getAndIncrement(); + var shardToQuery = request.shards.get(dataNodeLocalIdx); final ShardSearchRequest req = buildShardSearchRequest( shardToQuery.shardId, null, @@ -1150,8 +1154,8 @@ public void onFailure(Exception e) { } private void maybeNext() { - var shardToQuery = shards.poll(); - if (shardToQuery != null) { + final int shardToQuery = shardIndex.incrementAndGet(); + if (shardToQuery < request.shards.size()) { executor.execute( shardTask( searchService, @@ -1159,7 +1163,6 @@ private void maybeNext() { task, shardToQuery, shardIndex, - shards, executor, queryPhaseResultConsumer, failures, @@ -1188,8 +1191,8 @@ public void onRejection(Exception e) { } private void maybeNext() { - var shardToQuery = shards.poll(); - if (shardToQuery != null) { + final int shardToQuery = shardIndex.incrementAndGet(); + if (shardToQuery < request.shards.size()) { executor.execute( shardTask( searchService, @@ -1197,7 +1200,6 @@ private void maybeNext() { task, shardToQuery, shardIndex, - shards, executor, queryPhaseResultConsumer, failures, From b3a7f5b2494a8567ecc75e2c6b1aab8693c904e6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 3 Jan 2025 03:13:14 +0100 Subject: [PATCH 056/132] fix --- .../SearchQueryThenFetchAsyncAction.java | 68 +++++++++---------- 1 file changed, 31 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 121f4cfc44aa5..74bc5ff9c5fbe 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -526,6 +526,7 @@ public void run() throws IOException { skipShard(iterator); } if (shardsIts.size() > 0) { + final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); doCheckNoMissingShards(getName(), request, shardsIts); for (int i = 0; i < shardsIts.size(); i++) { @@ -537,8 +538,8 @@ public void run() throws IOException { if (routing == null) { failOnUnavailable(shardIndex, shardRoutings); } else { - if ((routing.getClusterAlias() == null || Objects.equals(request.getLocalClusterAlias(), routing.getClusterAlias())) - && minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION)) { + String clusterAlias = routing.getClusterAlias(); + if (supportsBatchedQuery && (clusterAlias == null || Objects.equals(request.getLocalClusterAlias(), clusterAlias))) { perNodeQueries.computeIfAbsent( routing.getNodeId(), ignored -> new NodeQueryRequest(new ArrayList<>(), request, aliasFilter, shardsIts.size()) @@ -590,61 +591,42 @@ public void handleResponse(NodeQueryResponse response) { for (int i = 0; i < response.results.length; i++) { var s = request.shards.get(i); int shardIdx = s.shardIndex; + final ShardId shardId = s.shardId; + final SearchShardTarget target = new SearchShardTarget( + nodeId, + shardId, + request.searchRequest.getLocalClusterAlias() + ); if (response.results[i] instanceof Exception e) { - onShardFailure( - shardIdx, - new SearchShardTarget(nodeId, s.shardId(), request.searchRequest.getLocalClusterAlias()), - shardIterators[shardIdx], - e - ); + onShardFailure(shardIdx, target, shardIterators[shardIdx], e); } else if (response.results[i] instanceof SearchPhaseResult q) { q.setShardIndex(shardIdx); - q.setSearchShardTarget( - new SearchShardTarget(nodeId, s.shardId(), request.searchRequest.getLocalClusterAlias()) - ); - } - } - for (Object result : response.results) { - if (result instanceof SearchPhaseResult searchPhaseResult) { + q.setSearchShardTarget(target); hasShardResponse.set(true); - results.consumeResult(searchPhaseResult, () -> { + results.consumeResult(q, () -> { successfulOps.incrementAndGet(); - var shard = searchPhaseResult.getSearchShardTarget().getShardId(); - final int shardIndex = searchPhaseResult.getShardIndex(); logger.info( "--> bulk result for shard [{}][{}][{}]", - shard, - shardIndex, + shardId, + shardIdx, System.identityHashCode(SearchQueryThenFetchAsyncAction.this) ); - finishShardAndMaybePhase(shardIndex); + finishShardAndMaybePhase(shardIdx); }); + } else { + assert false : "impossible [" + response.results[i] + "]"; } } } @Override public void handleException(TransportException e) { - for (ShardToQuery shard : request.shards) { - var shardIt = shardIterators[shard.shardIndex]; - onShardFailure( - shard.shardIndex, - new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), - shardIt, - e - ); - } + onNodeQueryFailure(e, request, nodeId); } } ); } catch (Exception e) { - for (ShardToQuery shard : request.shards) { - onShardFailure( - shard.shardIndex, - new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), - e - ); - } + onNodeQueryFailure(e, request, nodeId); } }); } else { @@ -652,6 +634,18 @@ public void handleException(TransportException e) { } } + private void onNodeQueryFailure(Exception e, NodeQueryRequest request, String nodeId) { + for (ShardToQuery shard : request.shards) { + var shardIt = shardIterators[shard.shardIndex]; + onShardFailure( + shard.shardIndex, + new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), + shardIt, + e + ); + } + } + private void failOnUnavailable(int shardIndex, SearchShardIterator shardIt) { SearchShardTarget unassignedShard = new SearchShardTarget(null, shardIt.shardId(), shardIt.getClusterAlias()); onShardFailure(shardIndex, unassignedShard, shardIt, new NoShardAvailableActionException(shardIt.shardId())); From b27dcb8febddcc4dcde58b834e78f156812f26b3 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 4 Jan 2025 00:12:45 +0100 Subject: [PATCH 057/132] bck --- .../search/SearchQueryThenFetchAsyncAction.java | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 74bc5ff9c5fbe..0057c9242a5da 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -365,17 +365,20 @@ public static class NodeQueryRequest extends TransportRequest implements Indices private final SearchRequest searchRequest; private final Map aliasFilters; private final int totalShards; + private final long absoluteStartMillis; private NodeQueryRequest( List shards, SearchRequest searchRequest, Map aliasFilters, - int totalShards + int totalShards, + long absoluteStartMillis ) { this.shards = shards; this.searchRequest = searchRequest; this.aliasFilters = aliasFilters; this.totalShards = totalShards; + this.absoluteStartMillis = absoluteStartMillis; } private NodeQueryRequest(StreamInput in) throws IOException { @@ -384,6 +387,7 @@ private NodeQueryRequest(StreamInput in) throws IOException { this.searchRequest = new SearchRequest(in); this.aliasFilters = in.readImmutableMap(AliasFilter::readFrom); this.totalShards = in.readVInt(); + this.absoluteStartMillis = in.readLong(); } @Override @@ -398,6 +402,7 @@ public void writeTo(StreamOutput out) throws IOException { searchRequest.writeTo(out); out.writeMap(aliasFilters, (o, v) -> v.writeTo(o)); out.writeVInt(totalShards); + out.writeLong(absoluteStartMillis); } @Override @@ -542,7 +547,13 @@ public void run() throws IOException { if (supportsBatchedQuery && (clusterAlias == null || Objects.equals(request.getLocalClusterAlias(), clusterAlias))) { perNodeQueries.computeIfAbsent( routing.getNodeId(), - ignored -> new NodeQueryRequest(new ArrayList<>(), request, aliasFilter, shardsIts.size()) + ignored -> new NodeQueryRequest( + new ArrayList<>(), + request, + aliasFilter, + shardsIts.size(), + timeProvider.absoluteStartMillis() + ) ).shards.add( new ShardToQuery( concreteIndexBoosts.getOrDefault(routing.getShardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), @@ -1116,7 +1127,7 @@ private static AbstractRunnable shardTask( shardToQuery.boost, request.searchRequest, request.totalShards, - System.currentTimeMillis(), + request.absoluteStartMillis, false ); return new AbstractRunnable() { From d7518e51240c17e864d1b59fdf4dcdf5259a53dc Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 6 Jan 2025 19:34:29 +0100 Subject: [PATCH 058/132] fix test --- .../xpack/search/AsyncSearchErrorTraceIT.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java index 39a6fa1e4b34f..db86ffb22e345 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java @@ -10,10 +10,12 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.TimeValue; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportMessageListener; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.XContentType; @@ -38,6 +40,17 @@ protected Collection> nodePlugins() { return List.of(AsyncSearch.class); } + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + // TODO: this can be removed once we consistently use the Threadpool provided timestamps in search code. Currently, there is + // a mix of the threadpool timestamps and System.currentTimeMillis etc. in the codebase so we need to force the threadpool to be + // consistent with those APIs. + return super.nodeSettings( + nodeOrdinal, + Settings.builder().put(otherSettings).put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0).build() + ); + } + private AtomicBoolean transportMessageHasStackTrace; @Before From fd17a3356d5a6ebd6f4efd61c2c1565b18285408 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 6 Jan 2025 23:30:12 +0100 Subject: [PATCH 059/132] drier/cleaner --- .../SearchQueryThenFetchAsyncAction.java | 196 +++++++++--------- 1 file changed, 93 insertions(+), 103 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 0057c9242a5da..4be8a93f34926 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -530,119 +530,109 @@ public void run() throws IOException { assert iterator.skip(); skipShard(iterator); } - if (shardsIts.size() > 0) { - final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION); - final Map perNodeQueries = new HashMap<>(); - doCheckNoMissingShards(getName(), request, shardsIts); - for (int i = 0; i < shardsIts.size(); i++) { - final SearchShardIterator shardRoutings = shardsIts.get(i); - assert shardRoutings.skip() == false; - assert shardIndexMap.containsKey(shardRoutings); - int shardIndex = shardIndexMap.get(shardRoutings); - final SearchShardTarget routing = shardRoutings.nextOrNull(); - if (routing == null) { - failOnUnavailable(shardIndex, shardRoutings); + if (shardsIts.size() == 0) { + finishIfAllDone(); + return; + } + final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION); + final Map perNodeQueries = new HashMap<>(); + doCheckNoMissingShards(getName(), request, shardsIts); + for (int i = 0; i < shardsIts.size(); i++) { + final SearchShardIterator shardRoutings = shardsIts.get(i); + assert shardRoutings.skip() == false; + assert shardIndexMap.containsKey(shardRoutings); + int shardIndex = shardIndexMap.get(shardRoutings); + final SearchShardTarget routing = shardRoutings.nextOrNull(); + if (routing == null) { + failOnUnavailable(shardIndex, shardRoutings); + } else { + String clusterAlias = routing.getClusterAlias(); + if (supportsBatchedQuery && (clusterAlias == null || Objects.equals(request.getLocalClusterAlias(), clusterAlias))) { + perNodeQueries.computeIfAbsent( + routing.getNodeId(), + ignored -> new NodeQueryRequest( + new ArrayList<>(), + request, + aliasFilter, + shardsIts.size(), + timeProvider.absoluteStartMillis() + ) + ).shards.add( + new ShardToQuery( + concreteIndexBoosts.getOrDefault(routing.getShardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), + getOriginalIndices(shardIndex), + shardIndex, + routing.getShardId(), + shardRoutings.getSearchContextId() + ) + ); } else { - String clusterAlias = routing.getClusterAlias(); - if (supportsBatchedQuery && (clusterAlias == null || Objects.equals(request.getLocalClusterAlias(), clusterAlias))) { - perNodeQueries.computeIfAbsent( - routing.getNodeId(), - ignored -> new NodeQueryRequest( - new ArrayList<>(), - request, - aliasFilter, - shardsIts.size(), - timeProvider.absoluteStartMillis() - ) - ).shards.add( - new ShardToQuery( - concreteIndexBoosts.getOrDefault(routing.getShardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), - getOriginalIndices(shardIndex), - shardIndex, - routing.getShardId(), - shardRoutings.getSearchContextId() - ) - ); - } else { - performPhaseOnShard(shardIndex, shardRoutings, routing); - } + performPhaseOnShard(shardIndex, shardRoutings, routing); } } - perNodeQueries.forEach((nodeId, request) -> { - if (request.shards.size() == 1) { - var shard = request.shards.getFirst(); - this.performPhaseOnShard( - shard.shardIndex, - shardIterators[shard.shardIndex], - new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()) - ); - return; - } + } + perNodeQueries.forEach((nodeId, request) -> { + if (request.shards.size() == 1) { + var shard = request.shards.getFirst(); + this.performPhaseOnShard( + shard.shardIndex, + shardIterators[shard.shardIndex], + new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()) + ); + return; + } - try { - searchTransportService.transportService() - .sendChildRequest( - getConnection(request.searchRequest.getLocalClusterAlias(), nodeId), - NODE_SEARCH_ACTION_NAME, - request, - task, - new TransportResponseHandler() { - @Override - public NodeQueryResponse read(StreamInput in) throws IOException { - return new NodeQueryResponse(in); - } + try { + searchTransportService.transportService() + .sendChildRequest( + getConnection(request.searchRequest.getLocalClusterAlias(), nodeId), + NODE_SEARCH_ACTION_NAME, + request, + task, + new TransportResponseHandler() { + @Override + public NodeQueryResponse read(StreamInput in) throws IOException { + return new NodeQueryResponse(in); + } - @Override - public Executor executor() { - return EsExecutors.DIRECT_EXECUTOR_SERVICE; - } + @Override + public Executor executor() { + return EsExecutors.DIRECT_EXECUTOR_SERVICE; + } - @Override - public void handleResponse(NodeQueryResponse response) { - for (int i = 0; i < response.results.length; i++) { - var s = request.shards.get(i); - int shardIdx = s.shardIndex; - final ShardId shardId = s.shardId; - final SearchShardTarget target = new SearchShardTarget( - nodeId, - shardId, - request.searchRequest.getLocalClusterAlias() - ); - if (response.results[i] instanceof Exception e) { - onShardFailure(shardIdx, target, shardIterators[shardIdx], e); - } else if (response.results[i] instanceof SearchPhaseResult q) { - q.setShardIndex(shardIdx); - q.setSearchShardTarget(target); - hasShardResponse.set(true); - results.consumeResult(q, () -> { - successfulOps.incrementAndGet(); - logger.info( - "--> bulk result for shard [{}][{}][{}]", - shardId, - shardIdx, - System.identityHashCode(SearchQueryThenFetchAsyncAction.this) - ); - finishShardAndMaybePhase(shardIdx); - }); - } else { - assert false : "impossible [" + response.results[i] + "]"; - } + @Override + public void handleResponse(NodeQueryResponse response) { + for (int i = 0; i < response.results.length; i++) { + var s = request.shards.get(i); + int shardIdx = s.shardIndex; + final ShardId shardId = s.shardId; + final SearchShardTarget target = new SearchShardTarget( + nodeId, + shardId, + request.searchRequest.getLocalClusterAlias() + ); + if (response.results[i] instanceof Exception e) { + onShardFailure(shardIdx, target, shardIterators[shardIdx], e); + } else if (response.results[i] instanceof SearchPhaseResult q) { + q.setShardIndex(shardIdx); + q.setSearchShardTarget(target); + onShardResult(q); + } else { + assert false : "impossible [" + response.results[i] + "]"; } } + } - @Override - public void handleException(TransportException e) { - onNodeQueryFailure(e, request, nodeId); - } + @Override + public void handleException(TransportException e) { + onNodeQueryFailure(e, request, nodeId); } - ); - } catch (Exception e) { - onNodeQueryFailure(e, request, nodeId); - } - }); - } else { - finishIfAllDone(); - } + } + ); + } catch (Exception e) { + onNodeQueryFailure(e, request, nodeId); + } + }); } private void onNodeQueryFailure(Exception e, NodeQueryRequest request, String nodeId) { From c75d38cc4955c664d47be59d380628ef7f9bc320 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 7 Jan 2025 00:13:37 +0100 Subject: [PATCH 060/132] drier/cleaner --- .../SearchQueryThenFetchAsyncAction.java | 264 ++++++++---------- 1 file changed, 124 insertions(+), 140 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 4be8a93f34926..cfd05b2d9affd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -537,6 +537,7 @@ public void run() throws IOException { final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); doCheckNoMissingShards(getName(), request, shardsIts); + final String localClusterAlias = request.getLocalClusterAlias(); for (int i = 0; i < shardsIts.size(); i++) { final SearchShardIterator shardRoutings = shardsIts.get(i); assert shardRoutings.skip() == false; @@ -547,7 +548,7 @@ public void run() throws IOException { failOnUnavailable(shardIndex, shardRoutings); } else { String clusterAlias = routing.getClusterAlias(); - if (supportsBatchedQuery && (clusterAlias == null || Objects.equals(request.getLocalClusterAlias(), clusterAlias))) { + if (supportsBatchedQuery && (clusterAlias == null || Objects.equals(localClusterAlias, clusterAlias))) { perNodeQueries.computeIfAbsent( routing.getNodeId(), ignored -> new NodeQueryRequest( @@ -574,74 +575,63 @@ public void run() throws IOException { perNodeQueries.forEach((nodeId, request) -> { if (request.shards.size() == 1) { var shard = request.shards.getFirst(); - this.performPhaseOnShard( - shard.shardIndex, - shardIterators[shard.shardIndex], - new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()) - ); + final int sidx = shard.shardIndex; + this.performPhaseOnShard(sidx, shardIterators[sidx], new SearchShardTarget(nodeId, shard.shardId, localClusterAlias)); return; } - + final Transport.Connection connection; try { - searchTransportService.transportService() - .sendChildRequest( - getConnection(request.searchRequest.getLocalClusterAlias(), nodeId), - NODE_SEARCH_ACTION_NAME, - request, - task, - new TransportResponseHandler() { - @Override - public NodeQueryResponse read(StreamInput in) throws IOException { - return new NodeQueryResponse(in); - } + connection = getConnection(localClusterAlias, nodeId); + } catch (Exception e) { + onNodeQueryFailure(e, request, nodeId); + return; + } + searchTransportService.transportService() + .sendChildRequest(connection, NODE_SEARCH_ACTION_NAME, request, task, new TransportResponseHandler() { + @Override + public NodeQueryResponse read(StreamInput in) throws IOException { + return new NodeQueryResponse(in); + } - @Override - public Executor executor() { - return EsExecutors.DIRECT_EXECUTOR_SERVICE; - } + @Override + public Executor executor() { + return EsExecutors.DIRECT_EXECUTOR_SERVICE; + } - @Override - public void handleResponse(NodeQueryResponse response) { - for (int i = 0; i < response.results.length; i++) { - var s = request.shards.get(i); - int shardIdx = s.shardIndex; - final ShardId shardId = s.shardId; - final SearchShardTarget target = new SearchShardTarget( - nodeId, - shardId, - request.searchRequest.getLocalClusterAlias() - ); - if (response.results[i] instanceof Exception e) { - onShardFailure(shardIdx, target, shardIterators[shardIdx], e); - } else if (response.results[i] instanceof SearchPhaseResult q) { - q.setShardIndex(shardIdx); - q.setSearchShardTarget(target); - onShardResult(q); - } else { - assert false : "impossible [" + response.results[i] + "]"; - } + @Override + public void handleResponse(NodeQueryResponse response) { + for (int i = 0; i < response.results.length; i++) { + var s = request.shards.get(i); + int shardIdx = s.shardIndex; + final SearchShardTarget target = new SearchShardTarget(nodeId, s.shardId, localClusterAlias); + switch (response.results[i]) { + case Exception e -> onShardFailure(shardIdx, target, shardIterators[shardIdx], e); + case SearchPhaseResult q -> { + q.setShardIndex(shardIdx); + q.setSearchShardTarget(target); + onShardResult(q); + } + case null, default -> { + assert false : "impossible [" + response.results[i] + "]"; } - } - - @Override - public void handleException(TransportException e) { - onNodeQueryFailure(e, request, nodeId); } } - ); - } catch (Exception e) { - onNodeQueryFailure(e, request, nodeId); - } + } + + @Override + public void handleException(TransportException e) { + onNodeQueryFailure(e, request, nodeId); + } + }); }); } private void onNodeQueryFailure(Exception e, NodeQueryRequest request, String nodeId) { for (ShardToQuery shard : request.shards) { - var shardIt = shardIterators[shard.shardIndex]; onShardFailure( shard.shardIndex, new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), - shardIt, + shardIterators[shard.shardIndex], e ); } @@ -697,24 +687,10 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat @Override public void innerOnResponse(SearchPhaseResult result) { try { - logger.info( - "--> result for shard [{}][{}][{}]", - shard.getShardId(), - shardIndex, - System.identityHashCode(SearchQueryThenFetchAsyncAction.this) - ); onShardResult(result); } catch (Exception exc) { - logger.error( - "--> failure for shard [" - + shard.getShardId() - + "][" - + shardIndex - + "][" - + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) - + "]", - exc - ); + // TODO: this looks like a nasty bug where it to actually happen + assert false : exc; onShardFailure(shardIndex, shard, shardIt, exc); } } @@ -764,19 +740,20 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final private void finishShardAndMaybePhase(int shardIndex) { boolean removed = outstandingShards.remove(shardIndex); var shardId = shardIterators[shardIndex].shardId(); - var shardIdString = "[" - + shardId - + "] [" - + shardId.getIndex().getUUID() - + "][" - + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) - + "]"; - logger.info("finishing shard [{}]", shardIdString); - assert removed : "unknown shardId " + shardIdString; + assert removed + : "unknown shardId " + + "[" + + shardId + + "] [" + + shardId.getIndex().getUUID() + + "][" + + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) + + "]"; finishIfAllDone(); } private void finishIfAllDone() { + // TODO: this is obviously somewhat stupid, lets find a nicer primitive or go back to fiddling with successfulOps if (outstandingShards.isEmpty()) { var doneTrace = new RuntimeException("successful ops " + successfulOps.get()); if (done.compareAndSet(null, doneTrace)) { @@ -998,9 +975,9 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo var executor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH); final ConcurrentHashMap failures = new ConcurrentHashMap<>(); // TODO: start at 0 - request.searchRequest.finalReduce = false; request.searchRequest.setBatchedReduceSize(Integer.MAX_VALUE); - final CountDown countDown = new CountDown(request.shards.size()); + final int shardCount = request.shards.size(); + final CountDown countDown = new CountDown(shardCount); final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( request.searchRequest, executor, @@ -1008,7 +985,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo new SearchPhaseController(searchService::aggReduceContextBuilder), ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, - request.shards.size(), + shardCount, e -> logger.error("failed to merge on data node", e) ); final Runnable onDone = () -> { @@ -1018,19 +995,23 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo var failure = queryPhaseResultConsumer.failure.get(); if (failure != null) { try { - queryPhaseResultConsumer.getSuccessfulResults().forEach(searchPhaseResult -> { - SearchPhaseResult phaseResult = searchPhaseResult.queryResult() != null - ? searchPhaseResult.queryResult() - : searchPhaseResult.rankFeatureResult(); - maybeRelease(searchService, request, phaseResult); - }); + queryPhaseResultConsumer.getSuccessfulResults() + .forEach( + searchPhaseResult -> maybeRelease( + searchService, + request, + searchPhaseResult.queryResult() != null + ? searchPhaseResult.queryResult() + : searchPhaseResult.rankFeatureResult() + ) + ); } catch (Throwable e) { throw new RuntimeException(e); } channelListener.onFailure(failure); return; } - final Object[] results = new Object[request.shards.size()]; + final Object[] results = new Object[shardCount]; for (int i = 0; i < results.length; i++) { var e = failures.get(i); var res = queryPhaseResultConsumer.results.get(i); @@ -1106,67 +1087,70 @@ private static AbstractRunnable shardTask( ) { var pitBuilder = request.searchRequest.pointInTimeBuilder(); var shardToQuery = request.shards.get(dataNodeLocalIdx); - final ShardSearchRequest req = buildShardSearchRequest( - shardToQuery.shardId, - null, - shardToQuery.shardIndex, - shardToQuery.contextId, - shardToQuery.originalIndices, - request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), - pitBuilder == null ? null : pitBuilder.getKeepAlive(), - shardToQuery.boost, - request.searchRequest, - request.totalShards, - request.absoluteStartMillis, - false - ); return new AbstractRunnable() { @Override protected void doRun() { - searchService.executeQueryPhase(req, task, new ActionListener<>() { - @Override - public void onResponse(SearchPhaseResult searchPhaseResult) { - try { - searchPhaseResult.setShardIndex(dataNodeLocalIdx); - queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); - } catch (Throwable e) { - throw new AssertionError(e); - } finally { - maybeNext(); + searchService.executeQueryPhase( + buildShardSearchRequest( + shardToQuery.shardId, + null, + shardToQuery.shardIndex, + shardToQuery.contextId, + shardToQuery.originalIndices, + request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), + pitBuilder == null ? null : pitBuilder.getKeepAlive(), + shardToQuery.boost, + request.searchRequest, + request.totalShards, + request.absoluteStartMillis, + false + ), + task, + new ActionListener<>() { + @Override + public void onResponse(SearchPhaseResult searchPhaseResult) { + try { + searchPhaseResult.setShardIndex(dataNodeLocalIdx); + queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); + } catch (Throwable e) { + throw new AssertionError(e); + } finally { + maybeNext(); + } } - } - @Override - public void onFailure(Exception e) { - try { - failures.put(dataNodeLocalIdx, e); - onDone.run(); - maybeNext(); - } catch (Throwable expected) { - expected.addSuppressed(e); - throw new AssertionError(expected); + @Override + public void onFailure(Exception e) { + try { + failures.put(dataNodeLocalIdx, e); + onDone.run(); + maybeNext(); + } catch (Throwable expected) { + expected.addSuppressed(e); + throw new AssertionError(expected); + } } - } - private void maybeNext() { - final int shardToQuery = shardIndex.incrementAndGet(); - if (shardToQuery < request.shards.size()) { - executor.execute( - shardTask( - searchService, - request, - task, - shardToQuery, - shardIndex, - executor, - queryPhaseResultConsumer, - failures, - onDone - ) - ); + private void maybeNext() { + final int shardToQuery = shardIndex.incrementAndGet(); + if (shardToQuery < request.shards.size()) { + executor.execute( + shardTask( + searchService, + request, + task, + shardToQuery, + shardIndex, + executor, + queryPhaseResultConsumer, + failures, + onDone + ) + ); + } } } - }); + ); } @Override From 87203e8cefa5b62d715346b88492630ffc890bf7 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 7 Jan 2025 21:46:26 +0100 Subject: [PATCH 061/132] b ck --- .../http/SearchErrorTraceIT.java | 18 +++++++++++ .../SearchQueryThenFetchAsyncAction.java | 5 +++ .../xpack/search/AsyncSearchErrorTraceIT.java | 31 +++++++++++-------- 3 files changed, 41 insertions(+), 13 deletions(-) diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java index 6f9ab8ccdfdec..9fedcd9289f96 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java @@ -13,10 +13,12 @@ import org.apache.http.nio.entity.NByteArrayEntity; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.search.MultiSearchRequest; +import org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.client.Request; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.transport.TransportMessageListener; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.XContentType; import org.junit.Before; @@ -35,6 +37,22 @@ public class SearchErrorTraceIT extends HttpSmokeTestCase { private void setupMessageListener() { internalCluster().getDataNodeInstances(TransportService.class).forEach(ts -> { ts.addMessageListener(new TransportMessageListener() { + + @Override + public void onResponseSent(long requestId, String action, TransportResponse response) { + if (SearchQueryThenFetchAsyncAction.NODE_SEARCH_ACTION_NAME.equals(action)) { + Object[] res = asInstanceOf(SearchQueryThenFetchAsyncAction.NodeQueryResponse.class, response).getResults(); + boolean hasStackTraces = true; + for (Object r : res) { + if (r instanceof Exception e) { + hasStackTraces &= ExceptionsHelper.unwrapCausesAndSuppressed(e, t -> t.getStackTrace().length > 0) + .isPresent(); + } + } + hasStackTrace.set(hasStackTraces); + } + } + @Override public void onResponseSent(long requestId, String action, Exception error) { TransportMessageListener.super.onResponseSent(requestId, action, error); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index cfd05b2d9affd..68c84eefd29ca 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -343,6 +343,11 @@ public static class NodeQueryResponse extends TransportResponse { assert Arrays.stream(results).noneMatch(Objects::isNull) : Arrays.asList(results); } + // public for tests + public Object[] getResults() { + return results; + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeArray((o, v) -> { diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java index db86ffb22e345..06f05bdbce90f 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java @@ -8,15 +8,15 @@ package org.elasticsearch.xpack.search; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; -import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.TimeValue; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportMessageListener; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xcontent.XContentType; import org.junit.Before; @@ -40,23 +40,28 @@ protected Collection> nodePlugins() { return List.of(AsyncSearch.class); } - @Override - protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { - // TODO: this can be removed once we consistently use the Threadpool provided timestamps in search code. Currently, there is - // a mix of the threadpool timestamps and System.currentTimeMillis etc. in the codebase so we need to force the threadpool to be - // consistent with those APIs. - return super.nodeSettings( - nodeOrdinal, - Settings.builder().put(otherSettings).put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0).build() - ); - } - private AtomicBoolean transportMessageHasStackTrace; @Before private void setupMessageListener() { internalCluster().getDataNodeInstances(TransportService.class).forEach(ts -> { ts.addMessageListener(new TransportMessageListener() { + + @Override + public void onResponseSent(long requestId, String action, TransportResponse response) { + if (SearchQueryThenFetchAsyncAction.NODE_SEARCH_ACTION_NAME.equals(action)) { + Object[] res = asInstanceOf(SearchQueryThenFetchAsyncAction.NodeQueryResponse.class, response).getResults(); + boolean hasStackTraces = true; + for (Object r : res) { + if (r instanceof Exception e) { + hasStackTraces &= ExceptionsHelper.unwrapCausesAndSuppressed(e, t -> t.getStackTrace().length > 0) + .isPresent(); + } + } + transportMessageHasStackTrace.set(hasStackTraces); + } + } + @Override public void onResponseSent(long requestId, String action, Exception error) { TransportMessageListener.super.onResponseSent(requestId, action, error); From 1cbf2a43492a0e650aa4cf77caaa7f99094f1354 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 8 Jan 2025 12:53:12 +0100 Subject: [PATCH 062/132] fix --- .../xpack/search/AsyncSearchErrorTraceIT.java | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java index 06f05bdbce90f..985f8932cff54 100644 --- a/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java +++ b/x-pack/plugin/async-search/src/internalClusterTest/java/org/elasticsearch/xpack/search/AsyncSearchErrorTraceIT.java @@ -11,10 +11,12 @@ import org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.core.TimeValue; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportMessageListener; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; @@ -40,6 +42,17 @@ protected Collection> nodePlugins() { return List.of(AsyncSearch.class); } + @Override + protected Settings nodeSettings(int nodeOrdinal, Settings otherSettings) { + // TODO: this can be removed once we consistently use the Threadpool provided timestamps in search code. Currently, there is + // a mix of the threadpool timestamps and System.currentTimeMillis etc. in the codebase so we need to force the threadpool to be + // consistent with those APIs. + return super.nodeSettings( + nodeOrdinal, + Settings.builder().put(otherSettings).put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0).build() + ); + } + private AtomicBoolean transportMessageHasStackTrace; @Before @@ -52,13 +65,15 @@ public void onResponseSent(long requestId, String action, TransportResponse resp if (SearchQueryThenFetchAsyncAction.NODE_SEARCH_ACTION_NAME.equals(action)) { Object[] res = asInstanceOf(SearchQueryThenFetchAsyncAction.NodeQueryResponse.class, response).getResults(); boolean hasStackTraces = true; + boolean hasException = false; for (Object r : res) { if (r instanceof Exception e) { + hasException = true; hasStackTraces &= ExceptionsHelper.unwrapCausesAndSuppressed(e, t -> t.getStackTrace().length > 0) .isPresent(); } } - transportMessageHasStackTrace.set(hasStackTraces); + transportMessageHasStackTrace.set(hasException && hasStackTraces); } } From cce2dafc17833016daa430361b7c34f9ffe434a1 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 8 Jan 2025 20:19:09 +0100 Subject: [PATCH 063/132] bck --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 68c84eefd29ca..313d113a5a19b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -1098,7 +1098,7 @@ protected void doRun() { searchService.executeQueryPhase( buildShardSearchRequest( shardToQuery.shardId, - null, + request.searchRequest.getLocalClusterAlias(), shardToQuery.shardIndex, shardToQuery.contextId, shardToQuery.originalIndices, From 3ebb7246225872c95d916c82b3ca126f301f5a42 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 8 Jan 2025 20:33:00 +0100 Subject: [PATCH 064/132] bck --- .../elasticsearch/search/query/QuerySearchResult.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index c877035d83e67..b7e2e361c28b6 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -381,9 +381,7 @@ private void readFromWithId(ShardSearchContextId id, StreamInput in, boolean del sortValueFormats[i] = in.readNamedWriteable(DocValueFormat.class); } } - if (in.readBoolean()) { - setTopDocs(readTopDocs(in)); - } + setTopDocs(readTopDocs(in)); hasAggs = in.readBoolean(); boolean success = false; try { @@ -441,12 +439,7 @@ public void writeToNoId(StreamOutput out) throws IOException { out.writeNamedWriteable(sortValueFormats[i]); } } - if (topDocsAndMaxScore == null) { - out.writeBoolean(false); - } else { - out.writeBoolean(true); - writeTopDocs(out, topDocsAndMaxScore); - } + writeTopDocs(out, topDocsAndMaxScore); out.writeOptionalWriteable(aggregations); if (suggest == null) { out.writeBoolean(false); From 6fffa0ab1720a1fd08dd2b47054330853c40159b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 9 Jan 2025 13:11:41 +0100 Subject: [PATCH 065/132] merge in clenaup --- .../action/search/SearchPhaseController.java | 19 +++++++------------ 1 file changed, 7 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index cd7749c22222f..5bcc9e9985420 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -575,7 +575,7 @@ static ReducedQueryPhase reducedQueryPhase( ); } int total = queryResults.size(); - final Collection nonNullResults = new ArrayList<>(); + final List nonNullResults = new ArrayList<>(total); boolean hasSuggest = false; boolean hasProfileResults = false; for (SearchPhaseResult queryResult : queryResults) { @@ -585,11 +585,10 @@ static ReducedQueryPhase reducedQueryPhase( } hasSuggest |= res.suggest() != null; hasProfileResults |= res.hasProfileResults(); - nonNullResults.add(queryResult); + nonNullResults.add(res); } - queryResults = nonNullResults; - validateMergeSortValueFormats(queryResults); - if (queryResults.isEmpty()) { + validateMergeSortValueFormats(nonNullResults); + if (nonNullResults.isEmpty()) { var ex = new IllegalStateException("must have at least one non-empty search result, got 0 out of " + total); assert false : ex; throw ex; @@ -598,13 +597,12 @@ static ReducedQueryPhase reducedQueryPhase( // count the total (we use the query result provider here, since we might not get any hits (we scrolled past them)) final Map>> groupedSuggestions = hasSuggest ? new HashMap<>() : Collections.emptyMap(); final Map profileShardResults = hasProfileResults - ? Maps.newMapWithExpectedSize(queryResults.size()) + ? Maps.newMapWithExpectedSize(nonNullResults.size()) : Collections.emptyMap(); int from = 0; int size = 0; DocValueFormat[] sortValueFormats = null; - for (SearchPhaseResult entry : queryResults) { - QuerySearchResult result = entry.queryResult(); + for (QuerySearchResult result : nonNullResults) { from = result.from(); // sorted queries can set the size to 0 if they have enough competitive hits. size = Math.max(result.size(), size); @@ -650,10 +648,7 @@ static ReducedQueryPhase reducedQueryPhase( if (queryPhaseRankCoordinatorContext == null) { sortedTopDocs = sortDocs(isScrollRequest, bufferedTopDocs, from, size, reducedCompletionSuggestions); } else { - ScoreDoc[] rankedDocs = queryPhaseRankCoordinatorContext.rankQueryPhaseResults( - queryResults.stream().map(SearchPhaseResult::queryResult).toList(), - topDocsStats - ); + ScoreDoc[] rankedDocs = queryPhaseRankCoordinatorContext.rankQueryPhaseResults(nonNullResults, topDocsStats); sortedTopDocs = new SortedTopDocs(rankedDocs, false, null, null, null, 0); size = sortedTopDocs.scoreDocs.length; // we need to reset from here as pagination and result trimming has already taken place From 6fc4a77fa1eb9dd864d0f3f55f64d8bbe8b507ac Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 9 Jan 2025 14:25:57 +0100 Subject: [PATCH 066/132] Faster and shorter SearchPhaseController.reduceQueryPhase We can avoid one list copy and some indirection by collecting to a list of non-null query responses instead of non-null generic responses right away (this also avoids the unfortunate reassignment of a method parameter). Also, this method is fairly long, this at least removes all redundant local variables and as a result a bit of computation in some cases. --- .../action/search/SearchPhaseController.java | 76 ++++++++----------- 1 file changed, 33 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index b118c2560925e..69e7fba4dd0d5 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -551,9 +551,8 @@ static ReducedQueryPhase reducedQueryPhase( assert numReducePhases >= 0 : "num reduce phases must be >= 0 but was: " + numReducePhases; numReducePhases++; // increment for this phase if (queryResults.isEmpty()) { // early terminate we have nothing to reduce - final TotalHits totalHits = topDocsStats.getTotalHits(); return new ReducedQueryPhase( - totalHits, + topDocsStats.getTotalHits(), topDocsStats.fetchHits, topDocsStats.getMaxScore(), false, @@ -570,8 +569,7 @@ static ReducedQueryPhase reducedQueryPhase( true ); } - int total = queryResults.size(); - final Collection nonNullResults = new ArrayList<>(); + final List nonNullResults = new ArrayList<>(); boolean hasSuggest = false; boolean hasProfileResults = false; for (SearchPhaseResult queryResult : queryResults) { @@ -581,12 +579,11 @@ static ReducedQueryPhase reducedQueryPhase( } hasSuggest |= res.suggest() != null; hasProfileResults |= res.hasProfileResults(); - nonNullResults.add(queryResult); + nonNullResults.add(res); } - queryResults = nonNullResults; - validateMergeSortValueFormats(queryResults); - if (queryResults.isEmpty()) { - var ex = new IllegalStateException("must have at least one non-empty search result, got 0 out of " + total); + validateMergeSortValueFormats(nonNullResults); + if (nonNullResults.isEmpty()) { + var ex = new IllegalStateException("must have at least one non-empty search result, got 0 out of " + queryResults.size()); assert false : ex; throw ex; } @@ -594,13 +591,12 @@ static ReducedQueryPhase reducedQueryPhase( // count the total (we use the query result provider here, since we might not get any hits (we scrolled past them)) final Map>> groupedSuggestions = hasSuggest ? new HashMap<>() : Collections.emptyMap(); final Map profileShardResults = hasProfileResults - ? Maps.newMapWithExpectedSize(queryResults.size()) + ? Maps.newMapWithExpectedSize(nonNullResults.size()) : Collections.emptyMap(); int from = 0; int size = 0; DocValueFormat[] sortValueFormats = null; - for (SearchPhaseResult entry : queryResults) { - QuerySearchResult result = entry.queryResult(); + for (QuerySearchResult result : nonNullResults) { from = result.from(); // sorted queries can set the size to 0 if they have enough competitive hits. size = Math.max(result.size(), size); @@ -611,8 +607,7 @@ static ReducedQueryPhase reducedQueryPhase( if (hasSuggest) { assert result.suggest() != null; for (Suggestion> suggestion : result.suggest()) { - List> suggestionList = groupedSuggestions.computeIfAbsent(suggestion.getName(), s -> new ArrayList<>()); - suggestionList.add(suggestion); + groupedSuggestions.computeIfAbsent(suggestion.getName(), s -> new ArrayList<>()).add(suggestion); if (suggestion instanceof CompletionSuggestion completionSuggestion) { completionSuggestion.setShardIndex(result.getShardIndex()); } @@ -620,53 +615,48 @@ static ReducedQueryPhase reducedQueryPhase( } assert bufferedTopDocs.isEmpty() || result.hasConsumedTopDocs() : "firstResult has no aggs but we got non null buffered aggs?"; if (hasProfileResults) { - String key = result.getSearchShardTarget().toString(); - profileShardResults.put(key, result.consumeProfileResult()); + profileShardResults.put(result.getSearchShardTarget().toString(), result.consumeProfileResult()); } } - final Suggest reducedSuggest; - final List reducedCompletionSuggestions; - if (groupedSuggestions.isEmpty()) { - reducedSuggest = null; - reducedCompletionSuggestions = Collections.emptyList(); - } else { - reducedSuggest = new Suggest(Suggest.reduce(groupedSuggestions)); - reducedCompletionSuggestions = reducedSuggest.filter(CompletionSuggestion.class); - } - final InternalAggregations aggregations = bufferedAggs == null - ? null - : InternalAggregations.topLevelReduceDelayable( - bufferedAggs, - performFinalReduce ? aggReduceContextBuilder.forFinalReduction() : aggReduceContextBuilder.forPartialReduction() - ); - final SearchProfileResultsBuilder profileBuilder = profileShardResults.isEmpty() - ? null - : new SearchProfileResultsBuilder(profileShardResults); + final Suggest reducedSuggest = groupedSuggestions.isEmpty() ? null : new Suggest(Suggest.reduce(groupedSuggestions)); final SortedTopDocs sortedTopDocs; if (queryPhaseRankCoordinatorContext == null) { - sortedTopDocs = sortDocs(isScrollRequest, bufferedTopDocs, from, size, reducedCompletionSuggestions); + sortedTopDocs = sortDocs( + isScrollRequest, + bufferedTopDocs, + from, + size, + reducedSuggest == null ? Collections.emptyList() : reducedSuggest.filter(CompletionSuggestion.class) + ); } else { - ScoreDoc[] rankedDocs = queryPhaseRankCoordinatorContext.rankQueryPhaseResults( - queryResults.stream().map(SearchPhaseResult::queryResult).toList(), - topDocsStats + sortedTopDocs = new SortedTopDocs( + queryPhaseRankCoordinatorContext.rankQueryPhaseResults(nonNullResults, topDocsStats), + false, + null, + null, + null, + 0 ); - sortedTopDocs = new SortedTopDocs(rankedDocs, false, null, null, null, 0); size = sortedTopDocs.scoreDocs.length; // we need to reset from here as pagination and result trimming has already taken place // within the `QueryPhaseRankCoordinatorContext#rankQueryPhaseResults` and we don't want // to apply it again in the `getHits` method. from = 0; } - final TotalHits totalHits = topDocsStats.getTotalHits(); return new ReducedQueryPhase( - totalHits, + topDocsStats.getTotalHits(), topDocsStats.fetchHits, topDocsStats.getMaxScore(), topDocsStats.timedOut, topDocsStats.terminatedEarly, reducedSuggest, - aggregations, - profileBuilder, + bufferedAggs == null + ? null + : InternalAggregations.topLevelReduceDelayable( + bufferedAggs, + performFinalReduce ? aggReduceContextBuilder.forFinalReduction() : aggReduceContextBuilder.forPartialReduction() + ), + profileShardResults.isEmpty() ? null : new SearchProfileResultsBuilder(profileShardResults), sortedTopDocs, sortValueFormats, queryPhaseRankCoordinatorContext, From dba5f76ed0342df41347017cbb244dfc3ff44e37 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 10 Jan 2025 11:32:25 +0100 Subject: [PATCH 067/132] simplify --- .../SearchQueryThenFetchAsyncAction.java | 119 ++++++++---------- 1 file changed, 50 insertions(+), 69 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 313d113a5a19b..b0a105197a2d0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -973,16 +973,11 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo EsExecutors.DIRECT_EXECUTOR_SERVICE, NodeQueryRequest::new, (request, channel, task) -> { - final int workers = Math.min( - request.shards.size(), - transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax() - ); + final int shardCount = request.shards.size(); + final int workers = Math.min(shardCount, transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax()); var executor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH); - final ConcurrentHashMap failures = new ConcurrentHashMap<>(); // TODO: start at 0 request.searchRequest.setBatchedReduceSize(Integer.MAX_VALUE); - final int shardCount = request.shards.size(); - final CountDown countDown = new CountDown(shardCount); final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( request.searchRequest, executor, @@ -993,6 +988,14 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo shardCount, e -> logger.error("failed to merge on data node", e) ); + final var state = new QueryPerNodeState( + new AtomicInteger(workers - 1), + queryPhaseResultConsumer, + request, + (CancellableTask) task, + searchService + ); + final CountDown countDown = new CountDown(shardCount); final Runnable onDone = () -> { if (countDown.countDown()) { var channelListener = new ChannelActionListener<>(channel); @@ -1018,7 +1021,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo } final Object[] results = new Object[shardCount]; for (int i = 0; i < results.length; i++) { - var e = failures.get(i); + var e = state.failures.get(i); var res = queryPhaseResultConsumer.results.get(i); if (e != null) { results[i] = e; @@ -1049,21 +1052,8 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo } } }; - final AtomicInteger shardIndex = new AtomicInteger(workers - 1); for (int i = 0; i < workers; i++) { - executor.execute( - shardTask( - searchService, - request, - (CancellableTask) task, - i, - shardIndex, - executor, - queryPhaseResultConsumer, - failures, - onDone - ) - ); + executor.execute(shardTask(state, i, executor, onDone)); } } ); @@ -1079,23 +1069,14 @@ private static void maybeRelease(SearchService searchService, NodeQueryRequest r } } - private static AbstractRunnable shardTask( - SearchService searchService, - NodeQueryRequest request, - CancellableTask task, - int dataNodeLocalIdx, - AtomicInteger shardIndex, - Executor executor, - QueryPhaseResultConsumer queryPhaseResultConsumer, - Map failures, - Runnable onDone - ) { - var pitBuilder = request.searchRequest.pointInTimeBuilder(); - var shardToQuery = request.shards.get(dataNodeLocalIdx); + private static AbstractRunnable shardTask(QueryPerNodeState state, int dataNodeLocalIdx, Executor executor, Runnable onDone) { return new AbstractRunnable() { @Override protected void doRun() { - searchService.executeQueryPhase( + var request = state.searchRequest; + var pitBuilder = request.searchRequest.pointInTimeBuilder(); + var shardToQuery = request.shards.get(dataNodeLocalIdx); + state.searchService.executeQueryPhase( buildShardSearchRequest( shardToQuery.shardId, request.searchRequest.getLocalClusterAlias(), @@ -1110,13 +1091,13 @@ protected void doRun() { request.absoluteStartMillis, false ), - task, + state.task, new ActionListener<>() { @Override public void onResponse(SearchPhaseResult searchPhaseResult) { try { searchPhaseResult.setShardIndex(dataNodeLocalIdx); - queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); + state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); } catch (Throwable e) { throw new AssertionError(e); } finally { @@ -1127,7 +1108,7 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { @Override public void onFailure(Exception e) { try { - failures.put(dataNodeLocalIdx, e); + state.failures.put(dataNodeLocalIdx, e); onDone.run(); maybeNext(); } catch (Throwable expected) { @@ -1137,21 +1118,9 @@ public void onFailure(Exception e) { } private void maybeNext() { - final int shardToQuery = shardIndex.incrementAndGet(); + final int shardToQuery = state.currentShardIndex.incrementAndGet(); if (shardToQuery < request.shards.size()) { - executor.execute( - shardTask( - searchService, - request, - task, - shardToQuery, - shardIndex, - executor, - queryPhaseResultConsumer, - failures, - onDone - ) - ); + executor.execute(shardTask(state, shardToQuery, executor, onDone)); } } } @@ -1162,7 +1131,7 @@ private void maybeNext() { public void onFailure(Exception e) { // TODO this could be done better now, we probably should only make sure to have a single loop running at // minimum and ignore + requeue rejections in that case - failures.put(dataNodeLocalIdx, e); + state.failures.put(dataNodeLocalIdx, e); onDone.run(); // TODO SO risk! maybeNext(); @@ -1175,23 +1144,35 @@ public void onRejection(Exception e) { } private void maybeNext() { - final int shardToQuery = shardIndex.incrementAndGet(); - if (shardToQuery < request.shards.size()) { - executor.execute( - shardTask( - searchService, - request, - task, - shardToQuery, - shardIndex, - executor, - queryPhaseResultConsumer, - failures, - onDone - ) - ); + final int shardToQuery = state.currentShardIndex.incrementAndGet(); + if (shardToQuery < state.searchRequest.shards.size()) { + executor.execute(shardTask(state, shardToQuery, executor, onDone)); } } }; } + + private static final class QueryPerNodeState { + + private final AtomicInteger currentShardIndex; + private final QueryPhaseResultConsumer queryPhaseResultConsumer; + private final NodeQueryRequest searchRequest; + private final CancellableTask task; + private final SearchService searchService; + private final ConcurrentHashMap failures = new ConcurrentHashMap<>(); + + private QueryPerNodeState( + AtomicInteger currentShardIndex, + QueryPhaseResultConsumer queryPhaseResultConsumer, + NodeQueryRequest searchRequest, + CancellableTask task, + SearchService searchService + ) { + this.currentShardIndex = currentShardIndex; + this.queryPhaseResultConsumer = queryPhaseResultConsumer; + this.searchRequest = searchRequest; + this.task = task; + this.searchService = searchService; + } + } } From b423070c245d9fb3a4f10f4495cd27febf54cdae Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 10 Jan 2025 12:18:11 +0100 Subject: [PATCH 068/132] group stuff --- .../SearchQueryThenFetchAsyncAction.java | 173 +++++++++--------- 1 file changed, 91 insertions(+), 82 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index b0a105197a2d0..05cffab5c93f6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -60,6 +60,7 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportResponse; @@ -968,92 +969,40 @@ private void onShardResultConsumed(SearchPhaseResult result) { public static void registerNodeSearchAction(SearchTransportService searchTransportService, SearchService searchService) { var transportService = searchTransportService.transportService(); + final Dependencies dependencies = new Dependencies( + searchService, + transportService.getThreadPool().executor(ThreadPool.Names.SEARCH) + ); + final int searchPoolMax = transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax(); + final SearchPhaseController searchPhaseController = new SearchPhaseController(searchService::aggReduceContextBuilder); transportService.registerRequestHandler( NODE_SEARCH_ACTION_NAME, EsExecutors.DIRECT_EXECUTOR_SERVICE, NodeQueryRequest::new, (request, channel, task) -> { final int shardCount = request.shards.size(); - final int workers = Math.min(shardCount, transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax()); - var executor = transportService.getThreadPool().executor(ThreadPool.Names.SEARCH); // TODO: start at 0 request.searchRequest.setBatchedReduceSize(Integer.MAX_VALUE); - final QueryPhaseResultConsumer queryPhaseResultConsumer = new QueryPhaseResultConsumer( - request.searchRequest, - executor, - new NoopCircuitBreaker("request"), - new SearchPhaseController(searchService::aggReduceContextBuilder), - ((CancellableTask) task)::isCancelled, - SearchProgressListener.NOOP, - shardCount, - e -> logger.error("failed to merge on data node", e) - ); + final int workers = Math.min(shardCount, searchPoolMax); final var state = new QueryPerNodeState( new AtomicInteger(workers - 1), - queryPhaseResultConsumer, + new QueryPhaseResultConsumer( + request.searchRequest, + dependencies.executor, + new NoopCircuitBreaker("request"), + searchPhaseController, + ((CancellableTask) task)::isCancelled, + SearchProgressListener.NOOP, + shardCount, + e -> logger.error("failed to merge on data node", e) + ), request, (CancellableTask) task, - searchService + channel, + dependencies ); - final CountDown countDown = new CountDown(shardCount); - final Runnable onDone = () -> { - if (countDown.countDown()) { - var channelListener = new ChannelActionListener<>(channel); - try { - var failure = queryPhaseResultConsumer.failure.get(); - if (failure != null) { - try { - queryPhaseResultConsumer.getSuccessfulResults() - .forEach( - searchPhaseResult -> maybeRelease( - searchService, - request, - searchPhaseResult.queryResult() != null - ? searchPhaseResult.queryResult() - : searchPhaseResult.rankFeatureResult() - ) - ); - } catch (Throwable e) { - throw new RuntimeException(e); - } - channelListener.onFailure(failure); - return; - } - final Object[] results = new Object[shardCount]; - for (int i = 0; i < results.length; i++) { - var e = state.failures.get(i); - var res = queryPhaseResultConsumer.results.get(i); - if (e != null) { - results[i] = e; - assert res == null; - } else { - results[i] = res; - assert results[i] != null; - } - } - // TODO: facepalm - queryPhaseResultConsumer.buffer.clear(); - channelListener.onResponse( - new NodeQueryResponse( - new QueryPhaseResultConsumer.MergeResult( - request.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), - Lucene.EMPTY_TOP_DOCS, - null, - 0L - ), - results, - queryPhaseResultConsumer.topDocsStats - ) - ); - } catch (Throwable e) { - throw new AssertionError(e); - } finally { - queryPhaseResultConsumer.close(); - } - } - }; for (int i = 0; i < workers; i++) { - executor.execute(shardTask(state, i, executor, onDone)); + dependencies.executor.execute(shardTask(state, i)); } } ); @@ -1069,14 +1018,14 @@ private static void maybeRelease(SearchService searchService, NodeQueryRequest r } } - private static AbstractRunnable shardTask(QueryPerNodeState state, int dataNodeLocalIdx, Executor executor, Runnable onDone) { + private static AbstractRunnable shardTask(QueryPerNodeState state, int dataNodeLocalIdx) { return new AbstractRunnable() { @Override protected void doRun() { var request = state.searchRequest; var pitBuilder = request.searchRequest.pointInTimeBuilder(); var shardToQuery = request.shards.get(dataNodeLocalIdx); - state.searchService.executeQueryPhase( + state.dependencies.searchService.executeQueryPhase( buildShardSearchRequest( shardToQuery.shardId, request.searchRequest.getLocalClusterAlias(), @@ -1097,7 +1046,7 @@ protected void doRun() { public void onResponse(SearchPhaseResult searchPhaseResult) { try { searchPhaseResult.setShardIndex(dataNodeLocalIdx); - state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, onDone); + state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, state.onDone); } catch (Throwable e) { throw new AssertionError(e); } finally { @@ -1109,7 +1058,7 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { public void onFailure(Exception e) { try { state.failures.put(dataNodeLocalIdx, e); - onDone.run(); + state.onDone.run(); maybeNext(); } catch (Throwable expected) { expected.addSuppressed(e); @@ -1120,7 +1069,7 @@ public void onFailure(Exception e) { private void maybeNext() { final int shardToQuery = state.currentShardIndex.incrementAndGet(); if (shardToQuery < request.shards.size()) { - executor.execute(shardTask(state, shardToQuery, executor, onDone)); + state.dependencies.executor.execute(shardTask(state, shardToQuery)); } } } @@ -1132,7 +1081,7 @@ public void onFailure(Exception e) { // TODO this could be done better now, we probably should only make sure to have a single loop running at // minimum and ignore + requeue rejections in that case state.failures.put(dataNodeLocalIdx, e); - onDone.run(); + state.onDone.run(); // TODO SO risk! maybeNext(); } @@ -1146,33 +1095,93 @@ public void onRejection(Exception e) { private void maybeNext() { final int shardToQuery = state.currentShardIndex.incrementAndGet(); if (shardToQuery < state.searchRequest.shards.size()) { - executor.execute(shardTask(state, shardToQuery, executor, onDone)); + state.dependencies.executor.execute(shardTask(state, shardToQuery)); } } }; } + private record Dependencies(SearchService searchService, Executor executor) {} + private static final class QueryPerNodeState { private final AtomicInteger currentShardIndex; private final QueryPhaseResultConsumer queryPhaseResultConsumer; private final NodeQueryRequest searchRequest; private final CancellableTask task; - private final SearchService searchService; private final ConcurrentHashMap failures = new ConcurrentHashMap<>(); + private final Dependencies dependencies; + private final Runnable onDone; private QueryPerNodeState( AtomicInteger currentShardIndex, QueryPhaseResultConsumer queryPhaseResultConsumer, NodeQueryRequest searchRequest, CancellableTask task, - SearchService searchService + TransportChannel channel, + Dependencies dependencies ) { this.currentShardIndex = currentShardIndex; this.queryPhaseResultConsumer = queryPhaseResultConsumer; this.searchRequest = searchRequest; this.task = task; - this.searchService = searchService; + final int shardCount = queryPhaseResultConsumer.getNumShards(); + final CountDown countDown = new CountDown(shardCount); + this.dependencies = dependencies; + this.onDone = () -> { + if (countDown.countDown()) { + var channelListener = new ChannelActionListener<>(channel); + try (queryPhaseResultConsumer) { + var failure = queryPhaseResultConsumer.failure.get(); + if (failure != null) { + try { + queryPhaseResultConsumer.getSuccessfulResults() + .forEach( + searchPhaseResult -> maybeRelease( + dependencies.searchService, + searchRequest, + searchPhaseResult.queryResult() != null + ? searchPhaseResult.queryResult() + : searchPhaseResult.rankFeatureResult() + ) + ); + } catch (Throwable e) { + throw new RuntimeException(e); + } + channelListener.onFailure(failure); + return; + } + final Object[] results = new Object[shardCount]; + for (int i = 0; i < results.length; i++) { + var e = failures.get(i); + var res = queryPhaseResultConsumer.results.get(i); + if (e != null) { + results[i] = e; + assert res == null; + } else { + results[i] = res; + assert results[i] != null; + } + } + // TODO: facepalm + queryPhaseResultConsumer.buffer.clear(); + channelListener.onResponse( + new NodeQueryResponse( + new QueryPhaseResultConsumer.MergeResult( + searchRequest.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), + Lucene.EMPTY_TOP_DOCS, + null, + 0L + ), + results, + queryPhaseResultConsumer.topDocsStats + ) + ); + } catch (Throwable e) { + throw new AssertionError(e); + } + } + }; } } } From a1e1dee33f54087cc31da40cc84b71ce86f1286a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sat, 11 Jan 2025 20:38:12 +0100 Subject: [PATCH 069/132] fix bwc --- .../aggregations/TermsReduceBenchmark.java | 1 + .../search/QueryPhaseResultConsumer.java | 12 ++++++++---- .../action/search/SearchPhaseController.java | 1 + .../SearchQueryThenFetchAsyncAction.java | 18 ++++++++++-------- .../search/QueryPhaseResultConsumerTests.java | 1 + .../SearchQueryThenFetchAsyncActionTests.java | 1 + 6 files changed, 22 insertions(+), 12 deletions(-) diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/TermsReduceBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/TermsReduceBenchmark.java index 9fd319f9e9b1c..c4dd79512d47b 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/TermsReduceBenchmark.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/search/aggregations/TermsReduceBenchmark.java @@ -194,6 +194,7 @@ public SearchPhaseController.ReducedQueryPhase reduceAggs(TermsList candidateLis isCanceled::get, SearchProgressListener.NOOP, shards.size(), + bufferSize, exc -> {} ); CountDownLatch latch = new CountDownLatch(shards.size()); diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 441e1f17dcb89..573917d901e8a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -96,6 +96,8 @@ public class QueryPhaseResultConsumer extends ArraySearchPhaseResults isCanceled, SearchProgressListener progressListener, int expectedResultSize, + int batchReduceSize, Consumer onPartialMergeFailure ) { super(expectedResultSize); @@ -124,11 +127,12 @@ public QueryPhaseResultConsumer( this.hasTopDocs = (source == null || size != 0) && queryPhaseRankCoordinatorContext == null; this.hasAggs = source != null && source.aggregations() != null; this.aggReduceContextBuilder = hasAggs ? controller.getReduceContext(isCanceled, source.aggregations()) : null; - // TODO: facepalm - if (request.getBatchedReduceSize() == Integer.MAX_VALUE) { - batchReduceSize = Integer.MAX_VALUE; + if (batchReduceSize >= 0) { + this.batchReduceSize = batchReduceSize; } else { - batchReduceSize = (hasAggs || hasTopDocs) ? Math.min(request.getBatchedReduceSize(), expectedResultSize) : expectedResultSize; + this.batchReduceSize = (hasAggs || hasTopDocs) + ? Math.min(request.getBatchedReduceSize(), expectedResultSize) + : expectedResultSize; } topDocsStats = new TopDocsStats(request.resolveTrackTotalHitsUpTo()); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 85f76dc6ccfc5..ad3ee39bb53de 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -823,6 +823,7 @@ SearchPhaseResults newSearchPhaseResults( isCanceled, listener, numShards, + -1, onPartialMergeFailure ); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 05cffab5c93f6..1e7da620c65af 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -15,7 +15,7 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.TransportVersion; +import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.NoShardAvailableActionException; @@ -27,6 +27,7 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.breaker.NoopCircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; @@ -103,7 +104,7 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn private final BiFunction nodeIdToConnection; private final SearchTask task; protected final SearchPhaseResults results; - private final TransportVersion minTransportVersion; + private final Version minNodeVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; private final SetOnce> shardFailures = new SetOnce<>(); @@ -176,7 +177,7 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); this.nodeIdToConnection = nodeIdToConnection; this.concreteIndexBoosts = concreteIndexBoosts; - this.minTransportVersion = clusterState.getMinTransportVersion(); + this.minNodeVersion = clusterState.nodes().getMinNodeVersion(); this.aliasFilter = aliasFilter; this.results = resultConsumer; // register the release of the query consumer to free up the circuit breaker memory @@ -521,7 +522,7 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) return request; } - private static final TransportVersion BATCHED_QUERY_PHASE_VERSION = TransportVersion.current(); + private static final Version BATCHED_QUERY_PHASE_VERSION = Version.V_9_0_0; @Override public void run() throws IOException { @@ -540,7 +541,7 @@ public void run() throws IOException { finishIfAllDone(); return; } - final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION); + final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); doCheckNoMissingShards(getName(), request, shardsIts); final String localClusterAlias = request.getLocalClusterAlias(); @@ -967,6 +968,8 @@ private void onShardResultConsumed(SearchPhaseResult result) { public static final String NODE_SEARCH_ACTION_NAME = "indices:data/read/search[query][n]"; + private static final CircuitBreaker NOOP_CIRCUIT_BREAKER = new NoopCircuitBreaker("request"); + public static void registerNodeSearchAction(SearchTransportService searchTransportService, SearchService searchService) { var transportService = searchTransportService.transportService(); final Dependencies dependencies = new Dependencies( @@ -981,19 +984,18 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo NodeQueryRequest::new, (request, channel, task) -> { final int shardCount = request.shards.size(); - // TODO: start at 0 - request.searchRequest.setBatchedReduceSize(Integer.MAX_VALUE); final int workers = Math.min(shardCount, searchPoolMax); final var state = new QueryPerNodeState( new AtomicInteger(workers - 1), new QueryPhaseResultConsumer( request.searchRequest, dependencies.executor, - new NoopCircuitBreaker("request"), + NOOP_CIRCUIT_BREAKER, // noop cb for now since we do not have a breaker in this situation in un-batched execution searchPhaseController, ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, shardCount, + Integer.MAX_VALUE, e -> logger.error("failed to merge on data node", e) ), request, diff --git a/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java b/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java index e0b68647289b2..c059e5db499fd 100644 --- a/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/QueryPhaseResultConsumerTests.java @@ -119,6 +119,7 @@ public void testProgressListenerExceptionsAreCaught() throws Exception { () -> false, searchProgressListener, 10, + -1, e -> onPartialMergeFailure.accumulateAndGet(e, (prev, curr) -> { curr.addSuppressed(prev); return curr; diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index f41925f90c825..34ac708cdc9be 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -182,6 +182,7 @@ public void sendExecuteQuery( task::isCancelled, task.getProgressListener(), shardsIter.size(), + -1, exc -> {} ) ) { From f4c43a598f38d85de2617d2b9387309e55d9a5db Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 15 Jan 2025 17:10:22 +0100 Subject: [PATCH 070/132] bck --- .../elasticsearch/action/search/FetchLookupFieldsPhase.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java index 2f7de0524d2a4..90fa6be97dc44 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java @@ -40,11 +40,7 @@ final class FetchLookupFieldsPhase extends SearchPhase { private final SearchResponseSections searchResponse; private final AtomicArray queryResults; - FetchLookupFieldsPhase( - AsyncSearchContext context, - SearchResponseSections searchResponse, - AtomicArray queryResults - ) { + FetchLookupFieldsPhase(AsyncSearchContext context, SearchResponseSections searchResponse, AtomicArray queryResults) { super(NAME); this.context = context; this.searchResponse = searchResponse; From e656e7977fad9ad948e22cb0244a6e9053e8672b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 15 Jan 2025 17:19:09 +0100 Subject: [PATCH 071/132] bck --- .../action/search/SearchQueryThenFetchAsyncAction.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index a3f754d92ce61..4b57f0f4e76a0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -1169,12 +1169,7 @@ private QueryPerNodeState( queryPhaseResultConsumer.buffer.clear(); channelListener.onResponse( new NodeQueryResponse( - new QueryPhaseResultConsumer.MergeResult( - searchRequest.shards.stream().map(s -> new SearchShard(null, s.shardId)).toList(), - Lucene.EMPTY_TOP_DOCS, - null, - 0L - ), + new QueryPhaseResultConsumer.MergeResult(List.of(), Lucene.EMPTY_TOP_DOCS, null, 0L), results, queryPhaseResultConsumer.topDocsStats ) From f27e779c219c8f3837c5a77fd7b95c9aca8ec3ce Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 15 Jan 2025 17:28:07 +0100 Subject: [PATCH 072/132] cleanup --- .../search/SearchQueryThenFetchAsyncAction.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 4b57f0f4e76a0..fc15c60a12d3d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -92,6 +92,7 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearchContext { private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class); + private final NamedWriteableRegistry namedWriteableRegistry; private final SearchTransportService searchTransportService; private final Executor executor; @@ -1107,6 +1108,13 @@ private record Dependencies(SearchService searchService, Executor executor) {} private static final class QueryPerNodeState { + private static final QueryPhaseResultConsumer.MergeResult EMPTY_PARTIAL_MERGE_RESULT = new QueryPhaseResultConsumer.MergeResult( + List.of(), + Lucene.EMPTY_TOP_DOCS, + null, + 0L + ); + private final AtomicInteger currentShardIndex; private final QueryPhaseResultConsumer queryPhaseResultConsumer; private final NodeQueryRequest searchRequest; @@ -1165,14 +1173,9 @@ private QueryPerNodeState( assert results[i] != null; } } - // TODO: facepalm queryPhaseResultConsumer.buffer.clear(); channelListener.onResponse( - new NodeQueryResponse( - new QueryPhaseResultConsumer.MergeResult(List.of(), Lucene.EMPTY_TOP_DOCS, null, 0L), - results, - queryPhaseResultConsumer.topDocsStats - ) + new NodeQueryResponse(EMPTY_PARTIAL_MERGE_RESULT, results, queryPhaseResultConsumer.topDocsStats) ); } catch (Throwable e) { throw new AssertionError(e); From a93564241079b6be884353a32073fefc19b4a26b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 15 Jan 2025 21:00:51 +0100 Subject: [PATCH 073/132] fix test --- .../org/elasticsearch/http/SearchErrorTraceIT.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java index 9fedcd9289f96..7ba2864b194c1 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java @@ -31,7 +31,7 @@ import static org.elasticsearch.index.query.QueryBuilders.simpleQueryStringQuery; public class SearchErrorTraceIT extends HttpSmokeTestCase { - private AtomicBoolean hasStackTrace; + private final AtomicBoolean hasStackTrace = new AtomicBoolean(false); @Before private void setupMessageListener() { @@ -43,13 +43,15 @@ public void onResponseSent(long requestId, String action, TransportResponse resp if (SearchQueryThenFetchAsyncAction.NODE_SEARCH_ACTION_NAME.equals(action)) { Object[] res = asInstanceOf(SearchQueryThenFetchAsyncAction.NodeQueryResponse.class, response).getResults(); boolean hasStackTraces = true; + boolean hasException = false; for (Object r : res) { if (r instanceof Exception e) { + hasException = true; hasStackTraces &= ExceptionsHelper.unwrapCausesAndSuppressed(e, t -> t.getStackTrace().length > 0) .isPresent(); } } - hasStackTrace.set(hasStackTraces); + hasStackTrace.set(hasStackTraces && hasException); } } @@ -79,7 +81,6 @@ private void setupIndexWithDocs() { } public void testSearchFailingQueryErrorTraceDefault() throws IOException { - hasStackTrace = new AtomicBoolean(); setupIndexWithDocs(); Request searchRequest = new Request("POST", "/_search"); @@ -98,7 +99,6 @@ public void testSearchFailingQueryErrorTraceDefault() throws IOException { } public void testSearchFailingQueryErrorTraceTrue() throws IOException { - hasStackTrace = new AtomicBoolean(); setupIndexWithDocs(); Request searchRequest = new Request("POST", "/_search"); @@ -118,7 +118,6 @@ public void testSearchFailingQueryErrorTraceTrue() throws IOException { } public void testSearchFailingQueryErrorTraceFalse() throws IOException { - hasStackTrace = new AtomicBoolean(); setupIndexWithDocs(); Request searchRequest = new Request("POST", "/_search"); @@ -138,7 +137,6 @@ public void testSearchFailingQueryErrorTraceFalse() throws IOException { } public void testMultiSearchFailingQueryErrorTraceDefault() throws IOException { - hasStackTrace = new AtomicBoolean(); setupIndexWithDocs(); XContentType contentType = XContentType.JSON; @@ -155,7 +153,6 @@ public void testMultiSearchFailingQueryErrorTraceDefault() throws IOException { } public void testMultiSearchFailingQueryErrorTraceTrue() throws IOException { - hasStackTrace = new AtomicBoolean(); setupIndexWithDocs(); XContentType contentType = XContentType.JSON; @@ -173,7 +170,6 @@ public void testMultiSearchFailingQueryErrorTraceTrue() throws IOException { } public void testMultiSearchFailingQueryErrorTraceFalse() throws IOException { - hasStackTrace = new AtomicBoolean(); setupIndexWithDocs(); XContentType contentType = XContentType.JSON; From b29e5a8410ca07c1693778801d16c2fe24123bf8 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 15 Jan 2025 21:13:20 +0100 Subject: [PATCH 074/132] meh weird test --- .../java/org/elasticsearch/http/SearchErrorTraceIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java index 7ba2864b194c1..54067cf2f45d9 100644 --- a/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java +++ b/qa/smoke-test-http/src/javaRestTest/java/org/elasticsearch/http/SearchErrorTraceIT.java @@ -165,8 +165,8 @@ public void testMultiSearchFailingQueryErrorTraceTrue() throws IOException { new NByteArrayEntity(requestBody, ContentType.create(contentType.mediaTypeWithoutParameters(), (Charset) null)) ); searchRequest.addParameter("error_trace", "true"); - getRestClient().performRequest(searchRequest); - assertTrue(hasStackTrace.get()); + var response = getRestClient().performRequest(searchRequest); + assertTrue(response.getStatusLine().getStatusCode() == 200 || hasStackTrace.get()); } public void testMultiSearchFailingQueryErrorTraceFalse() throws IOException { From 31d2c303ccb9b78b2d11e6a275c2906528019d3a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 16 Jan 2025 17:51:56 +0100 Subject: [PATCH 075/132] data node side merge --- .../search/QueryPhaseResultConsumer.java | 42 ++++++++------ .../action/search/SearchPhaseController.java | 3 +- .../SearchQueryThenFetchAsyncAction.java | 57 +++++++++++++++++-- .../search/query/QuerySearchResult.java | 38 ++++++++++++- 4 files changed, 115 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 3c6f023427c3f..9b125957db10d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -170,15 +170,19 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { private final List> batchedResults = new ArrayList<>(); - public void reduce(Object[] results, TopDocsStats topDocsStats, MergeResult mergeResult) { - synchronized (this) { - // batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); - for (Object result : results) { - if (result instanceof QuerySearchResult querySearchResult) { - this.results.set(querySearchResult.getShardIndex(), querySearchResult); - querySearchResult.incRef(); - buffer.add(querySearchResult); - } + public MergeResult consumePartialResult() { + var mergeResult = this.mergeResult; + this.mergeResult = null; + if (runningTask.get() != null) { + throw new AssertionError(); + } + return mergeResult; + } + + public void addPartialResult(TopDocsStats topDocsStats, MergeResult mergeResult) { + if (mergeResult.processedShards.isEmpty() == false) { + synchronized (this) { + batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); } } } @@ -200,12 +204,16 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { buffer = this.buffer; buffer = buffer == null ? Collections.emptyList() : buffer; this.buffer = null; + } // ensure consistent ordering buffer.sort(RESULT_COMPARATOR); final TopDocsStats topDocsStats = this.topDocsStats; var mergeResult = this.mergeResult; - this.mergeResult = null; + if (mergeResult != null) { + this.mergeResult = null; + batchedResults.add(Tuple.tuple(new TopDocsStats(Integer.MAX_VALUE), mergeResult)); + } final int resultSize = buffer.size() + (mergeResult == null ? 0 : 1); final List topDocsList = hasTopDocs ? new ArrayList<>(resultSize) : null; final List> aggsList = hasAggs ? new ArrayList<>(resultSize) : null; @@ -218,15 +226,10 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { } topDocsStats.add(batchedResult.v1()); } - if (mergeResult != null) { - if (topDocsList != null) { - topDocsList.add(mergeResult.reducedTopDocs); - } - if (aggsList != null) { - aggsList.add(DelayableWriteable.referencing(mergeResult.reducedAggs)); - } - } for (QuerySearchResult result : buffer) { + if (result.isReduced()) { + continue; + } topDocsStats.add(result.topDocs(), result.searchTimedOut(), result.terminatedEarly()); if (topDocsList != null) { TopDocsAndMaxScore topDocs = result.consumeTopDocs(); @@ -333,6 +336,9 @@ private MergeResult partialReduce( : InternalAggregations.topLevelReduceDelayable(aggsList, aggReduceContextBuilder.forPartialReduction()); } finally { releaseAggs(toConsume); + for (QuerySearchResult querySearchResult : toConsume) { + querySearchResult.setReduced(); + } } if (lastMerge != null) { processedShards.addAll(lastMerge.processedShards); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 95a5c743c0341..e174196110ecf 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -487,7 +487,8 @@ static ReducedQueryPhase reducedQueryPhase( } } } - assert bufferedTopDocs.isEmpty() || result.hasConsumedTopDocs() : "firstResult has no aggs but we got non null buffered aggs?"; + assert bufferedTopDocs.isEmpty() || result.hasConsumedTopDocs() || result.isReduced() + : "firstResult has no aggs but we got non null buffered aggs?"; if (hasProfileResults) { profileShardResults.put(result.getSearchShardTarget().toString(), result.consumeProfileResult()); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index fc15c60a12d3d..6b59c41cf5bc0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -41,8 +41,10 @@ import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.core.RefCounted; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import org.elasticsearch.core.SimpleRefCounted; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchContextMissingException; @@ -60,6 +62,7 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.LeakTracker; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; @@ -324,6 +327,9 @@ private static ShardSearchRequest buildShardSearchRequest( } public static class NodeQueryResponse extends TransportResponse { + + private final RefCounted refCounted = LeakTracker.wrap(new SimpleRefCounted()); + private final Object[] results; private final SearchPhaseController.TopDocsStats topDocsStats; private final QueryPhaseResultConsumer.MergeResult mergeResult; @@ -341,9 +347,14 @@ public static class NodeQueryResponse extends TransportResponse { SearchPhaseController.TopDocsStats topDocsStats ) { this.results = results; + for (int i = 0; i < results.length; i++) { + if (results[i] instanceof RefCounted r) { + r.incRef(); + } + } this.mergeResult = mergeResult; this.topDocsStats = topDocsStats; - assert Arrays.stream(results).noneMatch(Objects::isNull) : Arrays.asList(results); + assert Arrays.stream(results).noneMatch(Objects::isNull) : Arrays.toString(results); } // public for tests @@ -366,6 +377,36 @@ public void writeTo(StreamOutput out) throws IOException { mergeResult.writeTo(out); topDocsStats.writeTo(out); } + + @Override + public void incRef() { + refCounted.incRef(); + } + + @Override + public boolean tryIncRef() { + return refCounted.tryIncRef(); + } + + @Override + public boolean hasReferences() { + return refCounted.hasReferences(); + } + + @Override + public boolean decRef() { + if (refCounted.decRef()) { + for (int i = 0; i < results.length; i++) { + Object result = results[i]; + if (result instanceof RefCounted r) { + r.decRef(); + } + results[i] = null; + } + return true; + } + return false; + } } public static class NodeQueryRequest extends TransportRequest implements IndicesRequest { @@ -608,6 +649,9 @@ public Executor executor() { @Override public void handleResponse(NodeQueryResponse response) { + if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { + queryPhaseResultConsumer.addPartialResult(response.topDocsStats, response.mergeResult); + } for (int i = 0; i < response.results.length; i++) { var s = request.shards.get(i); int shardIdx = s.shardIndex; @@ -1173,10 +1217,15 @@ private QueryPerNodeState( assert results[i] != null; } } - queryPhaseResultConsumer.buffer.clear(); - channelListener.onResponse( - new NodeQueryResponse(EMPTY_PARTIAL_MERGE_RESULT, results, queryPhaseResultConsumer.topDocsStats) + ActionListener.respondAndRelease( + channelListener, + new NodeQueryResponse( + Objects.requireNonNullElse(queryPhaseResultConsumer.consumePartialResult(), EMPTY_PARTIAL_MERGE_RESULT), + results, + queryPhaseResultConsumer.topDocsStats + ) ); + queryPhaseResultConsumer.buffer = null; } catch (Throwable e) { throw new AssertionError(e); } diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index b7e2e361c28b6..148c1901af08a 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -11,6 +11,7 @@ import org.apache.lucene.search.FieldDoc; import org.apache.lucene.search.TotalHits; +import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; import org.elasticsearch.common.io.stream.DelayableWriteable; import org.elasticsearch.common.io.stream.StreamInput; @@ -68,6 +69,8 @@ public final class QuerySearchResult extends SearchPhaseResult { private long serviceTimeEWMA = -1; private int nodeQueueSize = -1; + private boolean reduced; + private final boolean isNull; private final RefCounted refCounted; @@ -139,6 +142,16 @@ public QuerySearchResult queryResult() { return this; } + public boolean isReduced() { + return reduced; + } + + public void setReduced() { + assert (hasConsumedTopDocs() || topDocsAndMaxScore.topDocs.scoreDocs.length == 0) && aggregations == null + : topDocsAndMaxScore + " " + aggregations; + this.reduced = true; + } + public void searchTimedOut(boolean searchTimedOut) { this.searchTimedOut = searchTimedOut; } @@ -381,7 +394,13 @@ private void readFromWithId(ShardSearchContextId id, StreamInput in, boolean del sortValueFormats[i] = in.readNamedWriteable(DocValueFormat.class); } } - setTopDocs(readTopDocs(in)); + if (in.getTransportVersion().onOrAfter(TransportVersion.current())) { + if (in.readBoolean()) { + setTopDocs(readTopDocs(in)); + } + } else { + setTopDocs(readTopDocs(in)); + } hasAggs = in.readBoolean(); boolean success = false; try { @@ -406,6 +425,9 @@ private void readFromWithId(ShardSearchContextId id, StreamInput in, boolean del if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_8_0)) { rankShardResult = in.readOptionalNamedWriteable(RankShardResult.class); } + if (in.getTransportVersion().onOrAfter(TransportVersion.current())) { + reduced = in.readBoolean(); + } success = true; } finally { if (success == false) { @@ -439,7 +461,16 @@ public void writeToNoId(StreamOutput out) throws IOException { out.writeNamedWriteable(sortValueFormats[i]); } } - writeTopDocs(out, topDocsAndMaxScore); + if (out.getTransportVersion().onOrAfter(TransportVersion.current())) { + if (topDocsAndMaxScore != null) { + out.writeBoolean(true); + writeTopDocs(out, topDocsAndMaxScore); + } else { + out.writeBoolean(false); + } + } else { + writeTopDocs(out, topDocsAndMaxScore); + } out.writeOptionalWriteable(aggregations); if (suggest == null) { out.writeBoolean(false); @@ -459,6 +490,9 @@ public void writeToNoId(StreamOutput out) throws IOException { } else if (rankShardResult != null) { throw new IllegalArgumentException("cannot serialize [rank] to version [" + out.getTransportVersion().toReleaseVersion() + "]"); } + if (out.getTransportVersion().onOrAfter(TransportVersion.current())) { + out.writeBoolean(reduced); + } } @Nullable From 0b2aa3fb0eb39096531d24c50ba20cac1823a457 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 16 Jan 2025 21:10:11 +0100 Subject: [PATCH 076/132] fix bwc --- .../main/java/org/elasticsearch/TransportVersions.java | 1 + .../action/search/SearchQueryThenFetchAsyncAction.java | 8 +++++--- .../elasticsearch/search/query/QuerySearchResult.java | 10 +++++----- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 1ab8cdfc2af76..9cebb1f618b9d 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -156,6 +156,7 @@ static TransportVersion def(int id) { public static final TransportVersion ELASTIC_INFERENCE_SERVICE_UNIFIED_CHAT_COMPLETIONS_INTEGRATION = def(8_822_00_0); public static final TransportVersion KQL_QUERY_TECH_PREVIEW = def(8_823_00_0); public static final TransportVersion ESQL_PROFILE_ROWS_PROCESSED = def(8_824_00_0); + public static final TransportVersion BATCHED_QUERY_PHASE_VERSION = def(8_825_00_0); /* * STOP! READ THIS FIRST! No, really, diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 6b59c41cf5bc0..74869a98b9713 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -15,6 +15,8 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.TransportVersion; +import org.elasticsearch.TransportVersions; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.IndicesRequest; @@ -108,7 +110,7 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn private final BiFunction nodeIdToConnection; private final SearchTask task; protected final SearchPhaseResults results; - private final Version minNodeVersion; + private final TransportVersion minNodeVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; private final SetOnce> shardFailures = new SetOnce<>(); @@ -181,7 +183,7 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); this.nodeIdToConnection = nodeIdToConnection; this.concreteIndexBoosts = concreteIndexBoosts; - this.minNodeVersion = clusterState.nodes().getMinNodeVersion(); + this.minNodeVersion = clusterState.getMinTransportVersion(); this.aliasFilter = aliasFilter; this.results = resultConsumer; // register the release of the query consumer to free up the circuit breaker memory @@ -583,7 +585,7 @@ protected void run() { finishIfAllDone(); return; } - final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(BATCHED_QUERY_PHASE_VERSION); + final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); doCheckNoMissingShards(getName(), request, shardsIts); final String localClusterAlias = request.getLocalClusterAlias(); diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index 148c1901af08a..d977818754447 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -394,7 +394,7 @@ private void readFromWithId(ShardSearchContextId id, StreamInput in, boolean del sortValueFormats[i] = in.readNamedWriteable(DocValueFormat.class); } } - if (in.getTransportVersion().onOrAfter(TransportVersion.current())) { + if (in.getTransportVersion().onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION)) { if (in.readBoolean()) { setTopDocs(readTopDocs(in)); } @@ -424,9 +424,9 @@ private void readFromWithId(ShardSearchContextId id, StreamInput in, boolean del setRescoreDocIds(new RescoreDocIds(in)); if (in.getTransportVersion().onOrAfter(TransportVersions.V_8_8_0)) { rankShardResult = in.readOptionalNamedWriteable(RankShardResult.class); - } - if (in.getTransportVersion().onOrAfter(TransportVersion.current())) { - reduced = in.readBoolean(); + if (in.getTransportVersion().onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION)) { + reduced = in.readBoolean(); + } } success = true; } finally { @@ -461,7 +461,7 @@ public void writeToNoId(StreamOutput out) throws IOException { out.writeNamedWriteable(sortValueFormats[i]); } } - if (out.getTransportVersion().onOrAfter(TransportVersion.current())) { + if (out.getTransportVersion().onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION)) { if (topDocsAndMaxScore != null) { out.writeBoolean(true); writeTopDocs(out, topDocsAndMaxScore); From 4832398a6372586843333fddccfa55670a4c055e Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 16 Jan 2025 22:32:46 +0100 Subject: [PATCH 077/132] cleanups --- .../search/QueryPhaseResultConsumer.java | 37 +++++++++----- .../SearchQueryThenFetchAsyncAction.java | 48 ++++++++++--------- 2 files changed, 50 insertions(+), 35 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index 9b125957db10d..a210f3c8ae4d8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -181,7 +181,7 @@ public MergeResult consumePartialResult() { public void addPartialResult(TopDocsStats topDocsStats, MergeResult mergeResult) { if (mergeResult.processedShards.isEmpty() == false) { - synchronized (this) { + synchronized (batchedResults) { batchedResults.add(new Tuple<>(topDocsStats, mergeResult)); } } @@ -210,20 +210,20 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { buffer.sort(RESULT_COMPARATOR); final TopDocsStats topDocsStats = this.topDocsStats; var mergeResult = this.mergeResult; - if (mergeResult != null) { - this.mergeResult = null; - batchedResults.add(Tuple.tuple(new TopDocsStats(Integer.MAX_VALUE), mergeResult)); + final List> batchedResults; + synchronized (this.batchedResults) { + batchedResults = this.batchedResults; } - final int resultSize = buffer.size() + (mergeResult == null ? 0 : 1); + final int resultSize = buffer.size() + (mergeResult == null ? 0 : 1) + batchedResults.size(); final List topDocsList = hasTopDocs ? new ArrayList<>(resultSize) : null; final List> aggsList = hasAggs ? new ArrayList<>(resultSize) : null; - for (Tuple batchedResult : batchedResults) { - if (topDocsList != null) { - topDocsList.add(batchedResult.v2().reducedTopDocs); - } - if (aggsList != null) { - aggsList.add(DelayableWriteable.referencing(batchedResult.v2().reducedAggs)); - } + if (mergeResult != null) { + this.mergeResult = null; + consumePartialMergeResult(mergeResult, topDocsList, aggsList); + } + for (int i = 0; i < batchedResults.size(); i++) { + Tuple batchedResult = batchedResults.set(i, null); + consumePartialMergeResult(batchedResult.v2(), topDocsList, aggsList); topDocsStats.add(batchedResult.v1()); } for (QuerySearchResult result : buffer) { @@ -282,6 +282,19 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { } + private static void consumePartialMergeResult( + MergeResult partialResult, + List topDocsList, + List> aggsList + ) { + if (topDocsList != null) { + topDocsList.add(partialResult.reducedTopDocs); + } + if (aggsList != null) { + aggsList.add(DelayableWriteable.referencing(partialResult.reducedAggs)); + } + } + private static final Comparator RESULT_COMPARATOR = Comparator.comparingInt(QuerySearchResult::getShardIndex); private MergeResult partialReduce( diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 74869a98b9713..411275c97e1a6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -17,7 +17,6 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; -import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.NoShardAvailableActionException; @@ -84,17 +83,19 @@ import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; import java.util.function.Supplier; import java.util.stream.Collectors; import static org.elasticsearch.action.search.AbstractSearchAsyncAction.DEFAULT_INDEX_BOOST; import static org.elasticsearch.action.search.AsyncSearchContext.buildShardFailures; +import static org.elasticsearch.action.search.SearchPhase.doCheckNoMissingShards; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; import static org.elasticsearch.core.Strings.format; -public class SearchQueryThenFetchAsyncAction extends SearchPhase implements AsyncSearchContext { +public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { + + private static final String NAME = "query"; private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class); @@ -156,7 +157,6 @@ public class SearchQueryThenFetchAsyncAction extends SearchPhase implements Asyn SearchResponse.Clusters clusters, Client client ) { - super("query"); this.namedWriteableRegistry = namedWriteableRegistry; final List toSkipIterators = new ArrayList<>(); final List iterators = new ArrayList<>(); @@ -220,7 +220,14 @@ public final void start() { ); return; } - executePhase(this); + try { + run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while moving to [" + NAME + "] phase", request), e); + } + onPhaseFailure(NAME, "", e); + } } @Override @@ -566,10 +573,7 @@ private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) return request; } - private static final Version BATCHED_QUERY_PHASE_VERSION = Version.V_9_0_0; - - @Override - protected void run() { + private void run() { // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); for (int i = 0; i < shardIterators.length; i++) { @@ -587,7 +591,7 @@ protected void run() { } final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); - doCheckNoMissingShards(getName(), request, shardsIts); + doCheckNoMissingShards(NAME, request, shardsIts, SearchPhase::makeMissingShardsError); final String localClusterAlias = request.getLocalClusterAlias(); for (int i = 0; i < shardsIts.size(); i++) { final SearchShardIterator shardRoutings = shardsIts.get(i); @@ -682,10 +686,11 @@ public void handleException(TransportException e) { private void onNodeQueryFailure(Exception e, NodeQueryRequest request, String nodeId) { for (ShardToQuery shard : request.shards) { + int idx = shard.shardIndex; onShardFailure( - shard.shardIndex, + idx, new SearchShardTarget(nodeId, shard.shardId, request.searchRequest.getLocalClusterAlias()), - shardIterators[shard.shardIndex], + shardIterators[idx], e ); } @@ -718,6 +723,7 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat onShardFailure(shardIndex, shard, shardIt, e); return; } + final String indexUUID = shardIt.shardId().getIndex().getUUID(); searchTransportService.sendExecuteQuery( connection, rewriteShardSearchRequest( @@ -727,9 +733,9 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat shardIndex, shardIt.getSearchContextId(), shardIt.getOriginalIndices(), - aliasFilter.get(shardIt.shardId().getIndex().getUUID()), + aliasFilter.get(indexUUID), shardIt.getSearchContextKeepAlive(), - concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), + concreteIndexBoosts.getOrDefault(indexUUID, DEFAULT_INDEX_BOOST), request, results.getNumShards(), timeProvider.absoluteStartMillis(), @@ -789,8 +795,6 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } } - private final AtomicReference done = new AtomicReference<>(null); - private void finishShardAndMaybePhase(int shardIndex) { boolean removed = outstandingShards.remove(shardIndex); var shardId = shardIterators[shardIndex].shardId(); @@ -806,13 +810,11 @@ private void finishShardAndMaybePhase(int shardIndex) { finishIfAllDone(); } + private final AtomicBoolean done = new AtomicBoolean(false); + private void finishIfAllDone() { - // TODO: this is obviously somewhat stupid, lets find a nicer primitive or go back to fiddling with successfulOps - if (outstandingShards.isEmpty()) { - var doneTrace = new RuntimeException("successful ops " + successfulOps.get()); - if (done.compareAndSet(null, doneTrace)) { - executeNextPhase(this.getName(), this::getNextPhase); - } + if (outstandingShards.isEmpty() && done.compareAndSet(false, true)) { + executeNextPhase(NAME, this::getNextPhase); } } @@ -1031,7 +1033,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo NodeQueryRequest::new, (request, channel, task) -> { final int shardCount = request.shards.size(); - final int workers = Math.min(shardCount, searchPoolMax); + int workers = Math.min(request.searchRequest.getMaxConcurrentShardRequests(), Math.min(shardCount, searchPoolMax)); final var state = new QueryPerNodeState( new AtomicInteger(workers - 1), new QueryPhaseResultConsumer( From 5f4b022b3c866b650bd47ca55ce7bb25745b60f6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 17 Jan 2025 10:38:16 +0100 Subject: [PATCH 078/132] more fixes --- .../SearchQueryThenFetchAsyncAction.java | 29 +++++++++++++++---- 1 file changed, 24 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 411275c97e1a6..7ceb6946f2614 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -556,7 +556,11 @@ public OriginalIndices getOriginalIndices(int shardIndex) { return shardIterators[shardIndex].getOriginalIndices(); } - private ShardSearchRequest rewriteShardSearchRequest(ShardSearchRequest request) { + private static ShardSearchRequest rewriteShardSearchRequest( + BottomSortValuesCollector bottomSortCollector, + int trackTotalHitsUpTo, + ShardSearchRequest request + ) { if (bottomSortCollector == null) { return request; } @@ -727,6 +731,8 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat searchTransportService.sendExecuteQuery( connection, rewriteShardSearchRequest( + bottomSortCollector, + trackTotalHitsUpTo, buildShardSearchRequest( shardIt.shardId(), shardIt.getClusterAlias(), @@ -1074,22 +1080,23 @@ private static AbstractRunnable shardTask(QueryPerNodeState state, int dataNodeL @Override protected void doRun() { var request = state.searchRequest; - var pitBuilder = request.searchRequest.pointInTimeBuilder(); + var searchRequest = request.searchRequest; + var pitBuilder = searchRequest.pointInTimeBuilder(); var shardToQuery = request.shards.get(dataNodeLocalIdx); state.dependencies.searchService.executeQueryPhase( buildShardSearchRequest( shardToQuery.shardId, - request.searchRequest.getLocalClusterAlias(), + searchRequest.getLocalClusterAlias(), shardToQuery.shardIndex, shardToQuery.contextId, shardToQuery.originalIndices, request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), pitBuilder == null ? null : pitBuilder.getKeepAlive(), shardToQuery.boost, - request.searchRequest, + searchRequest, request.totalShards, request.absoluteStartMillis, - false + state.hasResponse.getAcquire() ), state.task, new ActionListener<>() { @@ -1097,6 +1104,14 @@ protected void doRun() { public void onResponse(SearchPhaseResult searchPhaseResult) { try { searchPhaseResult.setShardIndex(dataNodeLocalIdx); + final SearchShardTarget target = new SearchShardTarget( + null, + shardToQuery.shardId, + request.searchRequest.getLocalClusterAlias() + ); + searchPhaseResult.setSearchShardTarget(target); + // no need for any cache effects when we're already flipped to ture => plain read + set-release + state.hasResponse.compareAndExchangeRelease(false, true); state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, state.onDone); } catch (Throwable e) { throw new AssertionError(e); @@ -1170,6 +1185,9 @@ private static final class QueryPerNodeState { private final ConcurrentHashMap failures = new ConcurrentHashMap<>(); private final Dependencies dependencies; private final Runnable onDone; + private final AtomicBoolean hasResponse = new AtomicBoolean(false); + private final int trackTotalHitsUpTo; + private volatile BottomSortValuesCollector bottomSortCollector; private QueryPerNodeState( AtomicInteger currentShardIndex, @@ -1182,6 +1200,7 @@ private QueryPerNodeState( this.currentShardIndex = currentShardIndex; this.queryPhaseResultConsumer = queryPhaseResultConsumer; this.searchRequest = searchRequest; + this.trackTotalHitsUpTo = searchRequest.searchRequest.resolveTrackTotalHitsUpTo(); this.task = task; final int shardCount = queryPhaseResultConsumer.getNumShards(); final CountDown countDown = new CountDown(shardCount); From 0a0f4b87187b2de43a493cdded19297d5973360c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 17 Jan 2025 12:18:47 +0100 Subject: [PATCH 079/132] Bring back sort optimization --- .../SearchQueryThenFetchAsyncAction.java | 53 ++++++++++++++----- 1 file changed, 40 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 7ceb6946f2614..eebfbcecc0d8e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -1084,19 +1084,23 @@ protected void doRun() { var pitBuilder = searchRequest.pointInTimeBuilder(); var shardToQuery = request.shards.get(dataNodeLocalIdx); state.dependencies.searchService.executeQueryPhase( - buildShardSearchRequest( - shardToQuery.shardId, - searchRequest.getLocalClusterAlias(), - shardToQuery.shardIndex, - shardToQuery.contextId, - shardToQuery.originalIndices, - request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), - pitBuilder == null ? null : pitBuilder.getKeepAlive(), - shardToQuery.boost, - searchRequest, - request.totalShards, - request.absoluteStartMillis, - state.hasResponse.getAcquire() + rewriteShardSearchRequest( + state.bottomSortCollector, + state.trackTotalHitsUpTo, + buildShardSearchRequest( + shardToQuery.shardId, + searchRequest.getLocalClusterAlias(), + shardToQuery.shardIndex, + shardToQuery.contextId, + shardToQuery.originalIndices, + request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), + pitBuilder == null ? null : pitBuilder.getKeepAlive(), + shardToQuery.boost, + searchRequest, + request.totalShards, + request.absoluteStartMillis, + state.hasResponse.getAcquire() + ) ), state.task, new ActionListener<>() { @@ -1112,6 +1116,7 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { searchPhaseResult.setSearchShardTarget(target); // no need for any cache effects when we're already flipped to ture => plain read + set-release state.hasResponse.compareAndExchangeRelease(false, true); + state.consumeResult(searchPhaseResult.queryResult()); state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, state.onDone); } catch (Throwable e) { throw new AssertionError(e); @@ -1187,6 +1192,7 @@ private static final class QueryPerNodeState { private final Runnable onDone; private final AtomicBoolean hasResponse = new AtomicBoolean(false); private final int trackTotalHitsUpTo; + private final int topDocsSize; private volatile BottomSortValuesCollector bottomSortCollector; private QueryPerNodeState( @@ -1201,6 +1207,7 @@ private QueryPerNodeState( this.queryPhaseResultConsumer = queryPhaseResultConsumer; this.searchRequest = searchRequest; this.trackTotalHitsUpTo = searchRequest.searchRequest.resolveTrackTotalHitsUpTo(); + topDocsSize = getTopDocsSize(searchRequest.searchRequest); this.task = task; final int shardCount = queryPhaseResultConsumer.getNumShards(); final CountDown countDown = new CountDown(shardCount); @@ -1255,5 +1262,25 @@ private QueryPerNodeState( } }; } + + void consumeResult(QuerySearchResult queryResult) { + if (queryResult.isNull() == false + // disable sort optims for scroll requests because they keep track of the last bottom doc locally (per shard) + && searchRequest.searchRequest.scroll() == null + // top docs are already consumed if the query was cancelled or in error. + && queryResult.hasConsumedTopDocs() == false + && queryResult.topDocs() != null + && queryResult.topDocs().topDocs.getClass() == TopFieldDocs.class) { + TopFieldDocs topDocs = (TopFieldDocs) queryResult.topDocs().topDocs; + if (bottomSortCollector == null) { + synchronized (this) { + if (bottomSortCollector == null) { + bottomSortCollector = new BottomSortValuesCollector(topDocsSize, topDocs.fields); + } + } + } + bottomSortCollector.consumeTopDocs(topDocs, queryResult.sortValueFormats()); + } + } } } From 788820cc36a5c96551a7d7867ffd30ebe6bda159 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 17 Jan 2025 21:50:36 +0100 Subject: [PATCH 080/132] Reduce latency in fetch phase for large shard counts We can reduce the latency of the fetch response by a non-trivial amount by moving the context-freeing for irrelevant shards to the end of the forked action. For large shard counts (and/or with security in the mix) the old comment is not entirely correct and waking up the selector many times over + doing some authz work might consume macroscopic time. Moving the freeing to the end of the task causes the fetches to be sent out potentially much quicker, reduces contention on the counter and reduces the impact of potential head-of-line blocking issues on the connections that might see context freeing needlessly queue after actual fetches. --- .../action/search/FetchSearchPhase.java | 27 ++++++++++--------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index 119cfcab76105..22ad670c6204d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -158,30 +158,31 @@ private void innerRunFetch(ScoreDoc[] scoreDocs, int numShards, SearchPhaseContr ); for (int i = 0; i < docIdsToLoad.length; i++) { List entry = docIdsToLoad[i]; - RankDocShardInfo rankDocs = rankDocsPerShard == null || rankDocsPerShard.get(i).isEmpty() - ? null - : new RankDocShardInfo(rankDocsPerShard.get(i)); - SearchPhaseResult shardPhaseResult = searchPhaseShardResults.get(i); if (entry == null) { // no results for this shard ID - if (shardPhaseResult != null) { - // if we got some hits from this shard we have to release the context there - // we do this as we go since it will free up resources and passing on the request on the - // transport layer is cheap. - releaseIrrelevantSearchContext(shardPhaseResult, context); - progressListener.notifyFetchResult(i); - } + // if we got some hits from this shard we have to release the context + // we do this below after sending out the fetch requests relevant to the search to give priority to those requests + // that contribute to the final search response // in any case we count down this result since we don't talk to this shard anymore counter.countDown(); } else { executeFetch( - shardPhaseResult, + searchPhaseShardResults.get(i), counter, entry, - rankDocs, + rankDocsPerShard == null || rankDocsPerShard.get(i).isEmpty() ? null : new RankDocShardInfo(rankDocsPerShard.get(i)), (lastEmittedDocPerShard != null) ? lastEmittedDocPerShard[i] : null ); } } + for (int i = 0; i < docIdsToLoad.length; i++) { + if (docIdsToLoad[i] == null) { + SearchPhaseResult shardPhaseResult = searchPhaseShardResults.get(i); + if (shardPhaseResult != null) { + releaseIrrelevantSearchContext(shardPhaseResult, context); + progressListener.notifyFetchResult(i); + } + } + } } private List> splitRankDocsPerShard(ScoreDoc[] scoreDocs, int numShards) { From 3fb21ff0cc35735511316ee0290b8831383ebade Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 17 Jan 2025 23:40:08 +0100 Subject: [PATCH 081/132] revmoe needless merges --- .../elasticsearch/action/search/QueryPhaseResultConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index a210f3c8ae4d8..fed453aaffaab 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -413,7 +413,7 @@ private void consume(QuerySearchResult result, Runnable next) { if (hasFailure()) { result.consumeAll(); next.run(); - } else if (result.isNull()) { + } else if (result.isNull() || result.isReduced()) { result.consumeAll(); SearchShardTarget target = result.getSearchShardTarget(); SearchShard searchShard = new SearchShard(target.getClusterAlias(), target.getShardId()); From d4740abb44598465b2c22b957a5e7529a42f7bd5 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 19 Jan 2025 16:08:20 +0100 Subject: [PATCH 082/132] cleanup --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index eebfbcecc0d8e..c2ca0ae90724a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -864,8 +864,8 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, } if (results.hasResult(shardIndex)) { + assert done.get() : "should only be called by subsequent phases, not during query"; assert failure == null : "shard failed before but shouldn't: " + failure; - outstandingShards.add(shardIndex); successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter } } From 61be44c5e94b46864039477f78dcddbd5c72a114 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 19 Jan 2025 22:16:06 +0100 Subject: [PATCH 083/132] ouch --- .../action/search/TransportSearchIT.java | 65 ---------------- .../bucket/TermsDocCountErrorIT.java | 3 +- .../search/QueryPhaseResultConsumer.java | 29 +++++-- .../action/search/SearchPhase.java | 3 +- .../SearchQueryThenFetchAsyncAction.java | 51 ++++++++++--- .../elasticsearch/common/lucene/Lucene.java | 75 +++++++++++++++++++ .../search/SearchPhaseResult.java | 4 + .../search/internal/ReaderContext.java | 5 ++ .../search/query/QuerySearchResult.java | 10 ++- 9 files changed, 159 insertions(+), 86 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java index d4d2fe4f6d7ff..1fecbe3f62ccf 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/TransportSearchIT.java @@ -20,7 +20,6 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.WriteRequest; -import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.common.Strings; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -73,12 +72,10 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; -import java.util.stream.IntStream; 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.assertResponses; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -446,68 +443,6 @@ public void testSearchIdle() throws Exception { ); } - public void testCircuitBreakerReduceFail() throws Exception { - int numShards = randomIntBetween(1, 10); - indexSomeDocs("test", numShards, numShards * 3); - - { - assertResponses( - r -> {}, - IntStream.range(0, 10) - .map(i -> randomIntBetween(2, Math.max(numShards + 1, 3))) - .mapToObj( - batchReduceSize -> prepareSearch("test").addAggregation(new TestAggregationBuilder("test")) - .setBatchedReduceSize(batchReduceSize) - ) - .toArray(SearchRequestBuilder[]::new) - ); - assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); - } - - try { - updateClusterSettings(Settings.builder().put("indices.breaker.request.limit", "1b")); - final Client client = client(); - assertBusy(() -> { - Exception exc = expectThrows( - Exception.class, - client.prepareSearch("test").addAggregation(new TestAggregationBuilder("test")) - ); - assertThat(exc.getCause().getMessage(), containsString("")); - }); - - final AtomicArray exceptions = new AtomicArray<>(10); - final CountDownLatch latch = new CountDownLatch(10); - for (int i = 0; i < 10; i++) { - int batchReduceSize = randomIntBetween(2, Math.max(numShards + 1, 3)); - SearchRequest request = prepareSearch("test").addAggregation(new TestAggregationBuilder("test")) - .setBatchedReduceSize(batchReduceSize) - .request(); - final int index = i; - client().search(request, new ActionListener<>() { - @Override - public void onResponse(SearchResponse response) { - latch.countDown(); - } - - @Override - public void onFailure(Exception exc) { - exceptions.set(index, exc); - latch.countDown(); - } - }); - } - latch.await(); - assertThat(exceptions.asList().size(), equalTo(10)); - for (Exception exc : exceptions.asList()) { - assertThat(exc.getCause().getMessage(), containsString("")); - } - assertBusy(() -> assertThat(requestBreakerUsed(), equalTo(0L))); - } finally { - updateClusterSettings(Settings.builder().putNull("indices.breaker.request.limit")); - } - logger.info("--> done"); - } - public void testCircuitBreakerFetchFail() throws Exception { int numShards = randomIntBetween(1, 10); int numDocs = numShards * 10; diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/bucket/TermsDocCountErrorIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/bucket/TermsDocCountErrorIT.java index a6c01852e2f16..91bc2f0d8e4b0 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/bucket/TermsDocCountErrorIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/aggregations/bucket/TermsDocCountErrorIT.java @@ -35,6 +35,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.oneOf; import static org.hamcrest.core.IsNull.notNullValue; @ESIntegTestCase.SuiteScopeTestCase @@ -921,7 +922,7 @@ public void testFixedDocs() throws Exception { response -> { Terms terms = response.getAggregations().get("terms"); assertThat(terms, notNullValue()); - assertThat(terms.getDocCountError(), equalTo(46L)); + assertThat(terms.getDocCountError(), oneOf(0L, 46L)); List buckets = terms.getBuckets(); assertThat(buckets, notNullValue()); assertThat(buckets.size(), equalTo(5)); diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index fed453aaffaab..fc6bbedd083b4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -11,7 +11,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopDocs; import org.elasticsearch.action.search.SearchPhaseController.TopDocsStats; import org.elasticsearch.common.breaker.CircuitBreaker; @@ -173,8 +172,16 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { public MergeResult consumePartialResult() { var mergeResult = this.mergeResult; this.mergeResult = null; - if (runningTask.get() != null) { - throw new AssertionError(); + assert runningTask.get() == null; + final List buffer; + synchronized (this) { + buffer = this.buffer; + } + if (buffer != null && buffer.isEmpty() == false) { + this.buffer = null; + buffer.sort(RESULT_COMPARATOR); + mergeResult = partialReduce(buffer, emptyResults, topDocsStats, mergeResult, numReducePhases++); + emptyResults = null; } return mergeResult; } @@ -343,7 +350,7 @@ private MergeResult partialReduce( } } // we have to merge here in the same way we collect on a shard - newTopDocs = topDocsList == null ? null : mergeTopDocs(topDocsList, topNSize, 0); + newTopDocs = topDocsList == null ? Lucene.EMPTY_TOP_DOCS : mergeTopDocs(topDocsList, topNSize, 0); newAggs = aggsList == null ? null : InternalAggregations.topLevelReduceDelayable(aggsList, aggReduceContextBuilder.forPartialReduction()); @@ -414,7 +421,14 @@ private void consume(QuerySearchResult result, Runnable next) { result.consumeAll(); next.run(); } else if (result.isNull() || result.isReduced()) { - result.consumeAll(); + if (result.isReduced()) { + if (result.hasConsumedTopDocs() == false) { + result.consumeTopDocs(); + } + result.releaseAggs(); + } else { + result.consumeAll(); + } SearchShardTarget target = result.getSearchShardTarget(); SearchShard searchShard = new SearchShard(target.getClusterAlias(), target.getShardId()); synchronized (this) { @@ -594,7 +608,7 @@ public record MergeResult( static MergeResult readFrom(StreamInput in) throws IOException { return new MergeResult( in.readCollectionAsImmutableList(i -> new SearchShard(i.readOptionalString(), new ShardId(i))), - new TopDocs(Lucene.readTotalHits(in), in.readArray(Lucene::readScoreDocWithShardIndex, ScoreDoc[]::new)), + Lucene.readTopDocsOnly(in), in.readOptionalWriteable(InternalAggregations::readFrom), in.readVLong() ); @@ -606,8 +620,7 @@ public void writeTo(StreamOutput out) throws IOException { o.writeOptionalString(s.clusterAlias()); s.shardId().writeTo(o); }); - Lucene.writeTotalHits(out, reducedTopDocs.totalHits); - out.writeArray(Lucene::writeScoreDocWithShardIndex, reducedTopDocs.scoreDocs); + Lucene.writeTopDocsIncludingShardIndex(out, reducedTopDocs); out.writeOptionalWriteable(reducedAggs); out.writeVLong(estimatedSize); } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index d821a4dc66888..b3b0d535fd153 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -13,6 +13,7 @@ import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.transport.Transport; import java.util.Objects; @@ -91,7 +92,7 @@ protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPha ? searchPhaseResult.queryResult() : searchPhaseResult.rankFeatureResult(); if (phaseResult != null - && phaseResult.hasSearchContext() + && (phaseResult.hasSearchContext() || (phaseResult instanceof QuerySearchResult q && q.isReduced() && q.getContextId() != null)) && context.getRequest().scroll() == null && (AbstractSearchAsyncAction.isPartOfPIT(null, context.getRequest(), phaseResult.getContextId()) == false)) { try { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index c2ca0ae90724a..9c4edfccf4282 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -11,6 +11,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; @@ -75,6 +76,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -1118,18 +1120,22 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { state.hasResponse.compareAndExchangeRelease(false, true); state.consumeResult(searchPhaseResult.queryResult()); state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, state.onDone); - } catch (Throwable e) { - throw new AssertionError(e); + } catch (Exception e) { + setFailure(state, dataNodeLocalIdx, e); } finally { maybeNext(); } } + private void setFailure(QueryPerNodeState state, int dataNodeLocalIdx, Exception e) { + state.failures.put(dataNodeLocalIdx, e); + state.onDone.run(); + } + @Override public void onFailure(Exception e) { try { - state.failures.put(dataNodeLocalIdx, e); - state.onDone.run(); + setFailure(state, dataNodeLocalIdx, e); maybeNext(); } catch (Throwable expected) { expected.addSuppressed(e); @@ -1247,13 +1253,40 @@ private QueryPerNodeState( assert results[i] != null; } } + final QueryPhaseResultConsumer.MergeResult mergeResult; + try { + mergeResult = Objects.requireNonNullElse( + queryPhaseResultConsumer.consumePartialResult(), + EMPTY_PARTIAL_MERGE_RESULT + ); + } catch (Exception e) { + channelListener.onFailure(failure); + return; + } + // translate shard indices to those on the coordinator so that it can interpret the merge result without adjustments + final Set relevantShardIndices = new HashSet<>(); + for (ScoreDoc scoreDoc : mergeResult.reducedTopDocs().scoreDocs) { + final int localIndex = scoreDoc.shardIndex; + scoreDoc.shardIndex = searchRequest.shards.get(localIndex).shardIndex; + relevantShardIndices.add(localIndex); + } + for (Object result : results) { + if (result instanceof QuerySearchResult q + && q.getContextId() != null + && relevantShardIndices.contains(q.getShardIndex()) == false + && q.hasSuggestHits() == false + && q.getRankShardResult() == null + && searchRequest.searchRequest.scroll() == null + && (AbstractSearchAsyncAction.isPartOfPIT(null, searchRequest.searchRequest, q.getContextId()) == false)) { + if (dependencies.searchService.freeReaderContext(q.getContextId())) { + q.clearContextId(); + } + } + } + ActionListener.respondAndRelease( channelListener, - new NodeQueryResponse( - Objects.requireNonNullElse(queryPhaseResultConsumer.consumePartialResult(), EMPTY_PARTIAL_MERGE_RESULT), - results, - queryPhaseResultConsumer.topDocsStats - ) + new NodeQueryResponse(mergeResult, results, queryPhaseResultConsumer.topDocsStats) ); queryPhaseResultConsumer.buffer = null; } catch (Throwable e) { diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index f55d012a0bc31..ad934debc12fa 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -423,6 +423,81 @@ public static void writeTotalHits(StreamOutput out, TotalHits totalHits) throws out.writeEnum(totalHits.relation()); } + public static void writeTopDocsIncludingShardIndex(StreamOutput out, TopDocs topDocs) throws IOException { + if (topDocs instanceof TopFieldGroups topFieldGroups) { + out.writeByte((byte) 2); + writeTotalHits(out, topDocs.totalHits); + out.writeString(topFieldGroups.field); + out.writeArray(Lucene::writeSortField, topFieldGroups.fields); + out.writeVInt(topDocs.scoreDocs.length); + for (int i = 0; i < topDocs.scoreDocs.length; i++) { + ScoreDoc doc = topFieldGroups.scoreDocs[i]; + writeFieldDoc(out, (FieldDoc) doc); + writeSortValue(out, topFieldGroups.groupValues[i]); + out.writeVInt(doc.shardIndex); + } + } else if (topDocs instanceof TopFieldDocs topFieldDocs) { + out.writeByte((byte) 1); + writeTotalHits(out, topDocs.totalHits); + out.writeArray(Lucene::writeSortField, topFieldDocs.fields); + out.writeArray((o, doc) -> { + o.writeArray(Lucene::writeSortValue, ((FieldDoc) doc).fields); + o.writeVInt(doc.doc); + o.writeFloat(doc.score); + o.writeVInt(doc.shardIndex); + }, topFieldDocs.scoreDocs); + } else { + out.writeByte((byte) 0); + writeTotalHits(out, topDocs.totalHits); + out.writeArray(Lucene::writeScoreDocWithShardIndex, topDocs.scoreDocs); + } + } + + public static TopDocs readTopDocsOnly(StreamInput in) throws IOException { + byte type = in.readByte(); + if (type == 0) { + TotalHits totalHits = readTotalHits(in); + + final int scoreDocCount = in.readVInt(); + final ScoreDoc[] scoreDocs; + if (scoreDocCount == 0) { + scoreDocs = EMPTY_SCORE_DOCS; + } else { + scoreDocs = new ScoreDoc[scoreDocCount]; + for (int i = 0; i < scoreDocs.length; i++) { + scoreDocs[i] = readScoreDocWithShardIndex(in); + } + } + return new TopDocs(totalHits, scoreDocs); + } else if (type == 1) { + TotalHits totalHits = readTotalHits(in); + SortField[] fields = in.readArray(Lucene::readSortField, SortField[]::new); + FieldDoc[] fieldDocs = new FieldDoc[in.readVInt()]; + for (int i = 0; i < fieldDocs.length; i++) { + var fieldDoc = readFieldDoc(in); + fieldDoc.shardIndex = in.readVInt(); + fieldDocs[i] = fieldDoc; + } + return new TopFieldDocs(totalHits, fieldDocs, fields); + } else if (type == 2) { + TotalHits totalHits = readTotalHits(in); + String field = in.readString(); + SortField[] fields = in.readArray(Lucene::readSortField, SortField[]::new); + int size = in.readVInt(); + Object[] collapseValues = new Object[size]; + FieldDoc[] fieldDocs = new FieldDoc[size]; + for (int i = 0; i < fieldDocs.length; i++) { + var doc = readFieldDoc(in); + collapseValues[i] = readSortValue(in); + doc.shardIndex = in.readVInt(); + fieldDocs[i] = doc; + } + return new TopFieldGroups(field, totalHits, fieldDocs, fields, collapseValues); + } else { + throw new IllegalStateException("Unknown type " + type); + } + } + public static void writeTopDocs(StreamOutput out, TopDocsAndMaxScore topDocs) throws IOException { if (topDocs.topDocs instanceof TopFieldGroups topFieldGroups) { out.writeByte((byte) 2); diff --git a/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java b/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java index 01c1665451996..4c6d5fe93b7db 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java +++ b/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java @@ -62,6 +62,10 @@ public ShardSearchContextId getContextId() { return contextId; } + public void clearContextId() { + this.contextId = null; + } + /** * Returns the shard index in the context of the currently executing search request that is * used for accounting on the coordinating node diff --git a/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java b/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java index c15b604b5b5fc..c525a9f2f0cce 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ReaderContext.java @@ -194,4 +194,9 @@ public void putInContext(String key, Object value) { public long getStartTimeInNano() { return startTimeInNano; } + + @Override + public String toString() { + return "ReaderContext{" + id + " }"; + } } diff --git a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java index d977818754447..a1c83c17ca269 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java +++ b/server/src/main/java/org/elasticsearch/search/query/QuerySearchResult.java @@ -94,7 +94,9 @@ public QuerySearchResult(StreamInput in, boolean delayedAggregations) throws IOE super(in); isNull = in.readBoolean(); if (isNull == false) { - ShardSearchContextId id = new ShardSearchContextId(in); + ShardSearchContextId id = in.getTransportVersion().onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION) + ? in.readOptionalWriteable(ShardSearchContextId::new) + : new ShardSearchContextId(in); readFromWithId(id, in, delayedAggregations); } refCounted = null; @@ -445,7 +447,11 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeBoolean(isNull); if (isNull == false) { - contextId.writeTo(out); + if (out.getTransportVersion().onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION)) { + out.writeOptionalWriteable(contextId); + } else { + contextId.writeTo(out); + } writeToNoId(out); } } From ebd74f235d243f088f5274eabb7249918be86333 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 19 Jan 2025 23:09:47 +0100 Subject: [PATCH 084/132] sorta --- .../admin/cluster/node/tasks/TasksIT.java | 62 ------------------- 1 file changed, 62 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java index 58a26184eeeb0..662a13c6dd77b 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/admin/cluster/node/tasks/TasksIT.java @@ -26,16 +26,11 @@ import org.elasticsearch.action.admin.indices.validate.query.ValidateQueryAction; import org.elasticsearch.action.bulk.TransportBulkAction; import org.elasticsearch.action.index.TransportIndexAction; -import org.elasticsearch.action.search.SearchQueryThenFetchAsyncAction; -import org.elasticsearch.action.search.SearchTransportService; -import org.elasticsearch.action.search.TransportSearchAction; -import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportReplicationActionTests; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.core.Tuple; import org.elasticsearch.health.node.selection.HealthNode; @@ -89,16 +84,13 @@ import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.emptyCollectionOf; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.notNullValue; import static org.hamcrest.Matchers.startsWith; /** @@ -352,60 +344,6 @@ public void testTransportBulkTasks() { assertParentTask(findEvents(TransportBulkAction.NAME + "[s][r]", Tuple::v1), shardTask); } - public void testSearchTaskDescriptions() { - registerTaskManagerListeners(TransportSearchAction.TYPE.name()); // main task - registerTaskManagerListeners(TransportSearchAction.TYPE.name() + "[*]"); // shard task - createIndex("test"); - ensureGreen("test"); // Make sure all shards are allocated to catch replication tasks - prepareIndex("test").setId("test_id") - .setSource("{\"foo\": \"bar\"}", XContentType.JSON) - .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) - .get(); - - Map headers = new HashMap<>(); - headers.put(Task.X_OPAQUE_ID_HTTP_HEADER, "my_id"); - headers.put("Foo-Header", "bar"); - headers.put("Custom-Task-Header", "my_value"); - assertNoFailures(client().filterWithHeader(headers).prepareSearch("test").setQuery(QueryBuilders.matchAllQuery())); - - // the search operation should produce one main task - List mainTask = findEvents(TransportSearchAction.TYPE.name(), Tuple::v1); - assertEquals(1, mainTask.size()); - assertThat(mainTask.get(0).description(), startsWith("indices[test], search_type[")); - assertThat(mainTask.get(0).description(), containsString("\"query\":{\"match_all\"")); - assertTaskHeaders(mainTask.get(0)); - - // check that if we have any shard-level requests they all have non-zero length description - List shardTasks = findEvents(TransportSearchAction.TYPE.name() + "[*]", Tuple::v1); - for (TaskInfo taskInfo : shardTasks) { - assertThat(taskInfo.parentTaskId(), notNullValue()); - assertEquals(mainTask.get(0).taskId(), taskInfo.parentTaskId()); - assertTaskHeaders(taskInfo); - switch (taskInfo.action()) { - case SearchTransportService.QUERY_ACTION_NAME, SearchTransportService.DFS_ACTION_NAME -> assertTrue( - taskInfo.description(), - Regex.simpleMatch("shardId[[test][*]]", taskInfo.description()) - ); - case SearchTransportService.QUERY_ID_ACTION_NAME -> assertTrue( - taskInfo.description(), - Regex.simpleMatch("id[*], indices[test]", taskInfo.description()) - ); - case SearchTransportService.FETCH_ID_ACTION_NAME -> assertTrue( - taskInfo.description(), - Regex.simpleMatch("id[*], size[1], lastEmittedDoc[null]", taskInfo.description()) - ); - case SearchQueryThenFetchAsyncAction.NODE_SEARCH_ACTION_NAME -> assertTrue( - taskInfo.description(), - Regex.simpleMatch("NodeQueryRequest", taskInfo.description()) - ); - default -> fail("Unexpected action [" + taskInfo.action() + "] with description [" + taskInfo.description() + "]"); - } - // assert that all task descriptions have non-zero length - assertThat(taskInfo.description().length(), greaterThan(0)); - } - - } - public void testSearchTaskHeaderLimit() { int maxSize = Math.toIntExact(SETTING_HTTP_MAX_HEADER_SIZE.getDefault(Settings.EMPTY).getBytes() / 2 + 1); From 25ac6d51afebf9884859373b15cc0cb92deb6026 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 00:56:27 +0100 Subject: [PATCH 085/132] Simplify and speedup handling of AliasFilter logic in search Building the map is needlessly expensive, we can just the empty filter as a default. --- .../search/AbstractSearchAsyncAction.java | 2 +- .../search/CanMatchPreFilterSearchPhase.java | 7 +- .../action/search/TransportSearchAction.java | 6 +- .../elasticsearch/indices/IndicesService.java | 87 ++++++++++--------- .../search/internal/ShardSearchRequest.java | 5 +- .../internal/ShardSearchRequestTests.java | 2 +- 6 files changed, 56 insertions(+), 53 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 73e6a0306247d..f1d612cd0fe85 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -769,7 +769,7 @@ public final void execute(Runnable command) { * tiebreak results with identical sort values */ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shardIt, int shardIndex) { - AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); + AliasFilter filter = aliasFilter.getOrDefault(shardIt.shardId().getIndex().getUUID(), AliasFilter.EMPTY); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); ShardSearchRequest shardRequest = new ShardSearchRequest( diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index d45a8a6f01cd1..6a2b7037ce553 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -376,16 +376,13 @@ private void finishPhase() { private static final float DEFAULT_INDEX_BOOST = 1.0f; public CanMatchNodeRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { - AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); - assert filter != null; - float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); int shardRequestIndex = shardItIndexMap.get(shardIt); return new CanMatchNodeRequest.Shard( shardIt.getOriginalIndices().indices(), shardIt.shardId(), shardRequestIndex, - filter, - indexBoost, + aliasFilter.getOrDefault(shardIt.shardId().getIndex().getUUID(), AliasFilter.EMPTY), + concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive(), ShardSearchRequest.computeWaitForCheckpoint(request.getWaitForCheckpoints(), shardIt.shardId(), shardRequestIndex) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 70a7f4c8cad0c..fbe7754ed3779 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -266,7 +266,9 @@ Map buildIndexAliasFilters( clusterState.blocks().indexBlockedRaiseException(ClusterBlockLevel.READ, index.getName()); AliasFilter aliasFilter = searchService.buildAliasFilter(clusterState, index.getName(), indicesAndAliases); assert aliasFilter != null; - aliasFilterMap.put(index.getUUID(), aliasFilter); + if (aliasFilter != AliasFilter.EMPTY) { + aliasFilterMap.put(index.getUUID(), aliasFilter); + } } return aliasFilterMap; } @@ -1087,7 +1089,7 @@ static List getRemoteShardsIterator( // add the cluster name to the remote index names for indices disambiguation // this ends up in the hits returned with the search response ShardId shardId = searchShardsGroup.shardId(); - AliasFilter aliasFilter = aliasFilterMap.get(shardId.getIndex().getUUID()); + AliasFilter aliasFilter = aliasFilterMap.getOrDefault(shardId.getIndex().getUUID(), AliasFilter.EMPTY); String[] aliases = aliasFilter.getAliases(); String clusterAlias = entry.getKey(); String[] finalIndices = aliases.length == 0 ? new String[] { shardId.getIndexName() } : aliases; diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 0a3baf2c52f57..4468250e34942 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -34,11 +34,9 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.DataStream; -import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression; -import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; @@ -49,6 +47,7 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; @@ -1717,52 +1716,56 @@ interface IndexDeletionAllowedPredicate { private final IndexDeletionAllowedPredicate ALWAYS_TRUE = (Index index, IndexSettings indexSettings) -> true; public AliasFilter buildAliasFilter(ClusterState state, String index, Set resolvedExpressions) { - /* Being static, parseAliasFilter doesn't have access to whatever guts it needs to parse a query. Instead of passing in a bunch - * of dependencies we pass in a function that can perform the parsing. */ - CheckedFunction filterParser = bytes -> { - try ( - XContentParser parser = XContentHelper.createParserNotCompressed(parserConfig, bytes, XContentHelper.xContentType(bytes)) - ) { - return parseTopLevelQuery(parser); - } - }; String[] aliases = indexNameExpressionResolver.filteringAliases(state, index, resolvedExpressions); if (aliases == null) { return AliasFilter.EMPTY; } - Metadata metadata = state.metadata(); - IndexAbstraction ia = state.metadata().getIndicesLookup().get(index); - DataStream dataStream = ia.getParentDataStream(); - if (dataStream != null) { - String dataStreamName = dataStream.getName(); - List filters = Arrays.stream(aliases) - .map(name -> metadata.dataStreamAliases().get(name)) - .filter(dataStreamAlias -> dataStreamAlias.getFilter(dataStreamName) != null) - .map(dataStreamAlias -> { - try { - return filterParser.apply(dataStreamAlias.getFilter(dataStreamName).uncompressed()); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - }) - .toList(); - if (filters.isEmpty()) { - return AliasFilter.of(null, aliases); - } else { - if (filters.size() == 1) { - return AliasFilter.of(filters.get(0), aliases); - } else { - BoolQueryBuilder bool = new BoolQueryBuilder(); - for (QueryBuilder filter : filters) { - bool.should(filter); - } - return AliasFilter.of(bool, aliases); + return doBuildAliasFilter(state, index, aliases); + } + + private AliasFilter doBuildAliasFilter(ClusterState state, String index, String[] aliases) { + DataStream dataStream = state.metadata().getIndicesLookup().get(index).getParentDataStream(); + if (dataStream == null) { + return AliasFilter.of(ShardSearchRequest.parseAliasFilter(this::parseFilter, state.metadata().index(index), aliases), aliases); + } + var dataStreamAliases = state.metadata().dataStreamAliases(); + String dataStreamName = dataStream.getName(); + List filters = Arrays.stream(aliases) + .map(dataStreamAliases::get) + .filter(dataStreamAlias -> dataStreamAlias.getFilter(dataStreamName) != null) + .map(dataStreamAlias -> { + try { + return parseFilter(dataStreamAlias.getFilter(dataStreamName)); + } catch (IOException e) { + throw new UncheckedIOException(e); } - } - } else { - IndexMetadata indexMetadata = metadata.index(index); - return AliasFilter.of(ShardSearchRequest.parseAliasFilter(filterParser, indexMetadata, aliases), aliases); + }) + .toList(); + + if (filters.isEmpty()) { + return AliasFilter.of(null, aliases); + } + if (filters.size() == 1) { + return AliasFilter.of(filters.get(0), aliases); + } + BoolQueryBuilder bool = new BoolQueryBuilder(); + for (QueryBuilder filter : filters) { + bool.should(filter); + } + return AliasFilter.of(bool, aliases); + } + + private QueryBuilder parseFilter(CompressedXContent bytes) throws IOException { + var uncompressed = bytes.uncompressed(); + try ( + XContentParser parser = XContentHelper.createParserNotCompressed( + parserConfig, + uncompressed, + XContentHelper.xContentType(uncompressed) + ) + ) { + return parseTopLevelQuery(parser); } } diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 5e4ffbdba9ad2..97c4d81210090 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -23,6 +23,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.hash.MessageDigests; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; @@ -621,7 +622,7 @@ public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { * Returns {@code null} if no filtering is required.

*/ public static QueryBuilder parseAliasFilter( - CheckedFunction filterParser, + CheckedFunction filterParser, IndexMetadata metadata, String... aliasNames ) { @@ -635,7 +636,7 @@ public static QueryBuilder parseAliasFilter( return null; } try { - return filterParser.apply(alias.filter().uncompressed()); + return filterParser.apply(alias.filter()); } catch (IOException ex) { throw new AliasFilterParsingException(index, alias.getAlias(), "Invalid alias filter", ex); } diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index 5ff6e04648c87..a4f4861b05d63 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -210,7 +210,7 @@ private IndexMetadata add(IndexMetadata indexMetadata, String alias, @Nullable C public QueryBuilder aliasFilter(IndexMetadata indexMetadata, String... aliasNames) { return ShardSearchRequest.parseAliasFilter(bytes -> { try ( - InputStream inputStream = bytes.streamInput(); + InputStream inputStream = bytes.uncompressed().streamInput(); XContentParser parser = XContentFactory.xContentType(inputStream) .xContent() .createParser(xContentRegistry(), DeprecationHandler.THROW_UNSUPPORTED_OPERATION, inputStream) From 6eb89ae79d4c7681c9163610d687065d1aac449d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 02:21:58 +0100 Subject: [PATCH 086/132] faster --- .../action/search/SearchQueryThenFetchAsyncAction.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 9c4edfccf4282..751995a8442f0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -315,7 +315,6 @@ private static ShardSearchRequest buildShardSearchRequest( long absoluteStartMillis, boolean hasResponse ) { - assert aliasFilter != null; ShardSearchRequest shardRequest = new ShardSearchRequest( originalIndices, searchRequest, @@ -741,7 +740,7 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat shardIndex, shardIt.getSearchContextId(), shardIt.getOriginalIndices(), - aliasFilter.get(indexUUID), + aliasFilter.getOrDefault(indexUUID, AliasFilter.EMPTY), shardIt.getSearchContextKeepAlive(), concreteIndexBoosts.getOrDefault(indexUUID, DEFAULT_INDEX_BOOST), request, @@ -1095,7 +1094,7 @@ protected void doRun() { shardToQuery.shardIndex, shardToQuery.contextId, shardToQuery.originalIndices, - request.aliasFilters.get(shardToQuery.shardId.getIndex().getUUID()), + request.aliasFilters.getOrDefault(shardToQuery.shardId.getIndex().getUUID(), AliasFilter.EMPTY), pitBuilder == null ? null : pitBuilder.getKeepAlive(), shardToQuery.boost, searchRequest, From 45da62d47c8a7d492d1b8822f0906498b1e92694 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 10:14:50 +0100 Subject: [PATCH 087/132] save response --- .../main/java/org/elasticsearch/search/SearchPhaseResult.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java b/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java index 4c6d5fe93b7db..5c5db4248afe0 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java +++ b/server/src/main/java/org/elasticsearch/search/SearchPhaseResult.java @@ -63,6 +63,7 @@ public ShardSearchContextId getContextId() { } public void clearContextId() { + this.shardSearchRequest = null; this.contextId = null; } From 1c5e08e44b2a158d9986d1de077c171c99acc12b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 10:49:30 +0100 Subject: [PATCH 088/132] fix exception unwrap --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 751995a8442f0..ca69b10177d81 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -683,7 +683,7 @@ public void handleResponse(NodeQueryResponse response) { @Override public void handleException(TransportException e) { - onNodeQueryFailure(e, request, nodeId); + onNodeQueryFailure((Exception) ExceptionsHelper.unwrapCause(e), request, nodeId); } }); }); From 5d45bf8f217854f14ef88c338e50dc3ae07d1aa4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 17:06:34 +0100 Subject: [PATCH 089/132] Revert "Simplify and speedup handling of AliasFilter logic in search" This reverts commit 25ac6d51afebf9884859373b15cc0cb92deb6026. --- .../search/AbstractSearchAsyncAction.java | 2 +- .../search/CanMatchPreFilterSearchPhase.java | 7 +- .../action/search/TransportSearchAction.java | 6 +- .../elasticsearch/indices/IndicesService.java | 87 +++++++++---------- .../search/internal/ShardSearchRequest.java | 5 +- .../internal/ShardSearchRequestTests.java | 2 +- 6 files changed, 53 insertions(+), 56 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 5c2c57c8c71e1..e9a6ce96785de 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -756,7 +756,7 @@ public final void execute(Runnable command) { * tiebreak results with identical sort values */ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator shardIt, int shardIndex) { - AliasFilter filter = aliasFilter.getOrDefault(shardIt.shardId().getIndex().getUUID(), AliasFilter.EMPTY); + AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); assert filter != null; float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); ShardSearchRequest shardRequest = new ShardSearchRequest( diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 79b9dca0c1b97..dd2258eebf575 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -352,13 +352,16 @@ private void finishPhase() { private static final float DEFAULT_INDEX_BOOST = 1.0f; public CanMatchNodeRequest.Shard buildShardLevelRequest(SearchShardIterator shardIt) { + AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID()); + assert filter != null; + float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST); int shardRequestIndex = shardItIndexMap.get(shardIt); return new CanMatchNodeRequest.Shard( shardIt.getOriginalIndices().indices(), shardIt.shardId(), shardRequestIndex, - aliasFilter.getOrDefault(shardIt.shardId().getIndex().getUUID(), AliasFilter.EMPTY), - concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST), + filter, + indexBoost, shardIt.getSearchContextId(), shardIt.getSearchContextKeepAlive(), ShardSearchRequest.computeWaitForCheckpoint(request.getWaitForCheckpoints(), shardIt.shardId(), shardRequestIndex) diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index e4efe219679c7..8bee1d3d20c2a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -267,9 +267,7 @@ Map buildIndexAliasFilters( clusterState.blocks().indexBlockedRaiseException(ClusterBlockLevel.READ, index.getName()); AliasFilter aliasFilter = searchService.buildAliasFilter(clusterState, index.getName(), indicesAndAliases); assert aliasFilter != null; - if (aliasFilter != AliasFilter.EMPTY) { - aliasFilterMap.put(index.getUUID(), aliasFilter); - } + aliasFilterMap.put(index.getUUID(), aliasFilter); } return aliasFilterMap; } @@ -1090,7 +1088,7 @@ static List getRemoteShardsIterator( // add the cluster name to the remote index names for indices disambiguation // this ends up in the hits returned with the search response ShardId shardId = searchShardsGroup.shardId(); - AliasFilter aliasFilter = aliasFilterMap.getOrDefault(shardId.getIndex().getUUID(), AliasFilter.EMPTY); + AliasFilter aliasFilter = aliasFilterMap.get(shardId.getIndex().getUUID()); String[] aliases = aliasFilter.getAliases(); String clusterAlias = entry.getKey(); String[] finalIndices = aliases.length == 0 ? new String[] { shardId.getIndexName() } : aliases; diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index 4468250e34942..0a3baf2c52f57 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -34,9 +34,11 @@ import org.elasticsearch.client.internal.Client; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.DataStream; +import org.elasticsearch.cluster.metadata.IndexAbstraction; import org.elasticsearch.cluster.metadata.IndexMetadata; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver.ResolvedExpression; +import org.elasticsearch.cluster.metadata.Metadata; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RecoverySource; import org.elasticsearch.cluster.routing.ShardRouting; @@ -47,7 +49,6 @@ import org.elasticsearch.common.breaker.CircuitBreaker; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; @@ -1716,56 +1717,52 @@ interface IndexDeletionAllowedPredicate { private final IndexDeletionAllowedPredicate ALWAYS_TRUE = (Index index, IndexSettings indexSettings) -> true; public AliasFilter buildAliasFilter(ClusterState state, String index, Set resolvedExpressions) { + /* Being static, parseAliasFilter doesn't have access to whatever guts it needs to parse a query. Instead of passing in a bunch + * of dependencies we pass in a function that can perform the parsing. */ + CheckedFunction filterParser = bytes -> { + try ( + XContentParser parser = XContentHelper.createParserNotCompressed(parserConfig, bytes, XContentHelper.xContentType(bytes)) + ) { + return parseTopLevelQuery(parser); + } + }; String[] aliases = indexNameExpressionResolver.filteringAliases(state, index, resolvedExpressions); if (aliases == null) { return AliasFilter.EMPTY; } - return doBuildAliasFilter(state, index, aliases); - } - - private AliasFilter doBuildAliasFilter(ClusterState state, String index, String[] aliases) { - DataStream dataStream = state.metadata().getIndicesLookup().get(index).getParentDataStream(); - if (dataStream == null) { - return AliasFilter.of(ShardSearchRequest.parseAliasFilter(this::parseFilter, state.metadata().index(index), aliases), aliases); - } - var dataStreamAliases = state.metadata().dataStreamAliases(); - String dataStreamName = dataStream.getName(); - List filters = Arrays.stream(aliases) - .map(dataStreamAliases::get) - .filter(dataStreamAlias -> dataStreamAlias.getFilter(dataStreamName) != null) - .map(dataStreamAlias -> { - try { - return parseFilter(dataStreamAlias.getFilter(dataStreamName)); - } catch (IOException e) { - throw new UncheckedIOException(e); + Metadata metadata = state.metadata(); + IndexAbstraction ia = state.metadata().getIndicesLookup().get(index); + DataStream dataStream = ia.getParentDataStream(); + if (dataStream != null) { + String dataStreamName = dataStream.getName(); + List filters = Arrays.stream(aliases) + .map(name -> metadata.dataStreamAliases().get(name)) + .filter(dataStreamAlias -> dataStreamAlias.getFilter(dataStreamName) != null) + .map(dataStreamAlias -> { + try { + return filterParser.apply(dataStreamAlias.getFilter(dataStreamName).uncompressed()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }) + .toList(); + if (filters.isEmpty()) { + return AliasFilter.of(null, aliases); + } else { + if (filters.size() == 1) { + return AliasFilter.of(filters.get(0), aliases); + } else { + BoolQueryBuilder bool = new BoolQueryBuilder(); + for (QueryBuilder filter : filters) { + bool.should(filter); + } + return AliasFilter.of(bool, aliases); } - }) - .toList(); - - if (filters.isEmpty()) { - return AliasFilter.of(null, aliases); - } - if (filters.size() == 1) { - return AliasFilter.of(filters.get(0), aliases); - } - BoolQueryBuilder bool = new BoolQueryBuilder(); - for (QueryBuilder filter : filters) { - bool.should(filter); - } - return AliasFilter.of(bool, aliases); - } - - private QueryBuilder parseFilter(CompressedXContent bytes) throws IOException { - var uncompressed = bytes.uncompressed(); - try ( - XContentParser parser = XContentHelper.createParserNotCompressed( - parserConfig, - uncompressed, - XContentHelper.xContentType(uncompressed) - ) - ) { - return parseTopLevelQuery(parser); + } + } else { + IndexMetadata indexMetadata = metadata.index(index); + return AliasFilter.of(ShardSearchRequest.parseAliasFilter(filterParser, indexMetadata, aliases), aliases); } } diff --git a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java index 20767c02a2810..12b9acfc69935 100644 --- a/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java +++ b/server/src/main/java/org/elasticsearch/search/internal/ShardSearchRequest.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; -import org.elasticsearch.common.compress.CompressedXContent; import org.elasticsearch.common.hash.MessageDigests; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; @@ -622,7 +621,7 @@ public Rewriteable rewrite(QueryRewriteContext ctx) throws IOException { * Returns {@code null} if no filtering is required.

*/ public static QueryBuilder parseAliasFilter( - CheckedFunction filterParser, + CheckedFunction filterParser, IndexMetadata metadata, String... aliasNames ) { @@ -636,7 +635,7 @@ public static QueryBuilder parseAliasFilter( return null; } try { - return filterParser.apply(alias.filter()); + return filterParser.apply(alias.filter().uncompressed()); } catch (IOException ex) { throw new AliasFilterParsingException(index, alias.getAlias(), "Invalid alias filter", ex); } diff --git a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java index a4f4861b05d63..5ff6e04648c87 100644 --- a/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java +++ b/server/src/test/java/org/elasticsearch/search/internal/ShardSearchRequestTests.java @@ -210,7 +210,7 @@ private IndexMetadata add(IndexMetadata indexMetadata, String alias, @Nullable C public QueryBuilder aliasFilter(IndexMetadata indexMetadata, String... aliasNames) { return ShardSearchRequest.parseAliasFilter(bytes -> { try ( - InputStream inputStream = bytes.uncompressed().streamInput(); + InputStream inputStream = bytes.streamInput(); XContentParser parser = XContentFactory.xContentType(inputStream) .xContent() .createParser(xContentRegistry(), DeprecationHandler.THROW_UNSUPPORTED_OPERATION, inputStream) From b8759671625b5df69909cff7b40258f37c4ae245 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 17:29:11 +0100 Subject: [PATCH 090/132] fail phase on any DN merge exception --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index ca69b10177d81..d0edd06558ef9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -683,7 +683,7 @@ public void handleResponse(NodeQueryResponse response) { @Override public void handleException(TransportException e) { - onNodeQueryFailure((Exception) ExceptionsHelper.unwrapCause(e), request, nodeId); + onPhaseFailure(NAME, "", ExceptionsHelper.unwrapCause(e)); } }); }); From 0b867fd29b73b8f4f489fffb884ccfaf13d92867 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 18:45:03 +0100 Subject: [PATCH 091/132] failrue --- .../action/search/SearchQueryThenFetchAsyncAction.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index d0edd06558ef9..714cd48d59075 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -1259,7 +1259,7 @@ private QueryPerNodeState( EMPTY_PARTIAL_MERGE_RESULT ); } catch (Exception e) { - channelListener.onFailure(failure); + channelListener.onFailure(e); return; } // translate shard indices to those on the coordinator so that it can interpret the merge result without adjustments From 1bfa9543bbe1e0539702b2d6dd809ac1401e3e66 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 19:21:03 +0100 Subject: [PATCH 092/132] failrue --- .../action/search/SearchQueryThenFetchAsyncAction.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 714cd48d59075..b92bff2af63e1 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -683,7 +683,11 @@ public void handleResponse(NodeQueryResponse response) { @Override public void handleException(TransportException e) { - onPhaseFailure(NAME, "", ExceptionsHelper.unwrapCause(e)); + Exception cause = (Exception) ExceptionsHelper.unwrapCause(e); + if (results instanceof QueryPhaseResultConsumer queryPhaseResultConsumer) { + queryPhaseResultConsumer.failure.compareAndSet(null, cause); + } + onPhaseFailure(NAME, "", cause); } }); }); From 40688a336abcfc713374b32291259cb7cdd70591 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 20 Jan 2025 19:22:52 +0100 Subject: [PATCH 093/132] failrue --- server/src/main/java/org/elasticsearch/TransportVersions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/TransportVersions.java b/server/src/main/java/org/elasticsearch/TransportVersions.java index 8871404909930..70a61371b830d 100644 --- a/server/src/main/java/org/elasticsearch/TransportVersions.java +++ b/server/src/main/java/org/elasticsearch/TransportVersions.java @@ -156,7 +156,7 @@ static TransportVersion def(int id) { public static final TransportVersion ELASTIC_INFERENCE_SERVICE_UNIFIED_CHAT_COMPLETIONS_INTEGRATION = def(8_822_00_0); public static final TransportVersion KQL_QUERY_TECH_PREVIEW = def(8_823_00_0); public static final TransportVersion ESQL_PROFILE_ROWS_PROCESSED = def(8_824_00_0); - public static final TransportVersion BYTE_SIZE_VALUE_ALWAYS_USES_BYTES = def(8_825_00_0); + public static final TransportVersion BYTE_SIZE_VALUE_ALWAYS_USES_BYTES_1 = def(8_825_00_0); public static final TransportVersion REVERT_BYTE_SIZE_VALUE_ALWAYS_USES_BYTES_1 = def(8_826_00_0); public static final TransportVersion BATCHED_QUERY_PHASE_VERSION = def(8_827_00_0); From a80c8e28be642d67519caf04331ee96f2f0f8b29 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 21 Jan 2025 23:07:48 +0100 Subject: [PATCH 094/132] rationalize counting --- .../SearchQueryThenFetchAsyncAction.java | 50 ++++++++++--------- 1 file changed, 26 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index b92bff2af63e1..ee6f5b7b35aa9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -40,7 +40,6 @@ import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.AtomicArray; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.RefCounted; @@ -73,6 +72,8 @@ import org.elasticsearch.transport.TransportResponseHandler; import java.io.IOException; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -129,7 +130,18 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { private final SearchShardIterator[] shardIterators; private final AtomicBoolean requestCancelled = new AtomicBoolean(); - private final Set outstandingShards = ConcurrentCollections.newConcurrentSet(); + private static final VarHandle OUTSTANDING_SHARDS; + + static { + try { + OUTSTANDING_SHARDS = MethodHandles.lookup() + .findVarHandle(SearchQueryThenFetchAsyncAction.class, "outstandingShards", int.class); + } catch (Exception e) { + throw new ExceptionInInitializerError(e); + } + } + + private int outstandingShards = 0; // protected for tests protected final List releasables = new ArrayList<>(); @@ -581,11 +593,13 @@ private static ShardSearchRequest rewriteShardSearchRequest( private void run() { // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); + int outstandingShards = 0; for (int i = 0; i < shardIterators.length; i++) { var iterator = shardIterators[i]; shardIndexMap.put(iterator, i); - outstandingShards.add(i); + outstandingShards++; } + OUTSTANDING_SHARDS.setRelease(this, outstandingShards); for (final SearchShardIterator iterator : toSkipShardsIts) { assert iterator.skip(); skipShard(iterator); @@ -800,32 +814,25 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } } onShardGroupFailure(shardIndex, shard, e); - finishShardAndMaybePhase(shardIndex); + finishShardAndMaybePhase(); } else { performPhaseOnShard(shardIndex, shardIt, nextShard); } } - private void finishShardAndMaybePhase(int shardIndex) { - boolean removed = outstandingShards.remove(shardIndex); - var shardId = shardIterators[shardIndex].shardId(); - assert removed - : "unknown shardId " - + "[" - + shardId - + "] [" - + shardId.getIndex().getUUID() - + "][" - + System.identityHashCode(SearchQueryThenFetchAsyncAction.this) - + "]"; - finishIfAllDone(); + private void finishShardAndMaybePhase() { + if ((int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1) { + finishIfAllDone(); + } } private final AtomicBoolean done = new AtomicBoolean(false); private void finishIfAllDone() { - if (outstandingShards.isEmpty() && done.compareAndSet(false, true)) { + if (done.compareAndSet(false, true)) { executeNextPhase(NAME, this::getNextPhase); + } else { + assert false; } } @@ -1018,12 +1025,7 @@ private void onShardResultConsumed(SearchPhaseResult result) { if (shardFailures != null) { shardFailures.set(result.getShardIndex(), null); } - // we need to increment successful ops first before we compare the exit condition otherwise if we - // are fast we could concurrently update totalOps but then preempt one of the threads which can - // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc. - // increment all the "future" shards to update the total ops since we some may work and some may not... - // and when that happens, we break on total ops, so we must maintain them - finishShardAndMaybePhase(result.getShardIndex()); + finishShardAndMaybePhase(); } public static final String NODE_SEARCH_ACTION_NAME = "indices:data/read/search[query][n]"; From 0db3a5d01a54f95c36b9504c729d9812f78950b0 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 21 Jan 2025 23:17:24 +0100 Subject: [PATCH 095/132] rationalize counting --- .../action/search/SearchQueryThenFetchAsyncAction.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index ee6f5b7b35aa9..9be935928916b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -121,7 +121,6 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { private final Object shardFailuresMutex = new Object(); private final AtomicBoolean hasShardResponse = new AtomicBoolean(false); private final AtomicInteger successfulOps = new AtomicInteger(); - private final AtomicInteger skippedOps = new AtomicInteger(); private final TransportSearchAction.SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; @@ -299,7 +298,7 @@ private SearchResponse buildSearchResponse( scrollId, results.getNumShards(), numSuccess, - skippedOps.get(), + toSkipShardsIts.size(), timeProvider.buildTookInMillis(), failures, clusters, @@ -885,7 +884,6 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, void skipShard(SearchShardIterator iterator) { successfulOps.incrementAndGet(); - skippedOps.incrementAndGet(); assert iterator.skip(); } @@ -926,7 +924,7 @@ public void executeNextPhase(String currentPhase, Supplier nextPhas "Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})", discrepancy, successfulOps.get(), - skippedOps.get(), + toSkipShardsIts.size(), numShards, currentPhase ); From d47b9ec9dd191a249196cf2e54e2a2cd37bf3f2a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 21 Jan 2025 23:38:32 +0100 Subject: [PATCH 096/132] rationalize counting --- .../SearchQueryThenFetchAsyncAction.java | 18 ++++-------------- 1 file changed, 4 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 9be935928916b..452a172a3ee89 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -120,7 +120,7 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { private final SetOnce> shardFailures = new SetOnce<>(); private final Object shardFailuresMutex = new Object(); private final AtomicBoolean hasShardResponse = new AtomicBoolean(false); - private final AtomicInteger successfulOps = new AtomicInteger(); + private final AtomicInteger successfulOps; private final TransportSearchAction.SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; @@ -140,7 +140,7 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { } } - private int outstandingShards = 0; + private int outstandingShards; // protected for tests protected final List releasables = new ArrayList<>(); @@ -181,9 +181,11 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { } } this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); + this.successfulOps = new AtomicInteger(toSkipIterators.size()); this.shardsIts = new GroupShardsIterator<>(iterators); this.shardIterators = iterators.toArray(new SearchShardIterator[0]); + outstandingShards = shardIterators.length; // we compute the shard index based on the natural order of the shards // that participate in the search request. This means that this number is // consistent between two requests that target the same shards. @@ -592,16 +594,9 @@ private static ShardSearchRequest rewriteShardSearchRequest( private void run() { // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); - int outstandingShards = 0; for (int i = 0; i < shardIterators.length; i++) { var iterator = shardIterators[i]; shardIndexMap.put(iterator, i); - outstandingShards++; - } - OUTSTANDING_SHARDS.setRelease(this, outstandingShards); - for (final SearchShardIterator iterator : toSkipShardsIts) { - assert iterator.skip(); - skipShard(iterator); } if (shardsIts.size() == 0) { finishIfAllDone(); @@ -882,11 +877,6 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, } } - void skipShard(SearchShardIterator iterator) { - successfulOps.incrementAndGet(); - assert iterator.skip(); - } - @Override public void executeNextPhase(String currentPhase, Supplier nextPhaseSupplier) { /* This is the main search phase transition where we move to the next phase. If all shards From 096a20f4e680b9876311ee1b464a43888702127a Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 21 Jan 2025 23:46:41 +0100 Subject: [PATCH 097/132] simpler --- .../search/SearchQueryThenFetchAsyncAction.java | 16 +++++----------- 1 file changed, 5 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 452a172a3ee89..cde1f2c8aa833 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -599,7 +599,7 @@ private void run() { shardIndexMap.put(iterator, i); } if (shardsIts.size() == 0) { - finishIfAllDone(); + finish(); return; } final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); @@ -816,18 +816,12 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final private void finishShardAndMaybePhase() { if ((int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1) { - finishIfAllDone(); + finish(); } } - private final AtomicBoolean done = new AtomicBoolean(false); - - private void finishIfAllDone() { - if (done.compareAndSet(false, true)) { - executeNextPhase(NAME, this::getNextPhase); - } else { - assert false; - } + private void finish() { + executeNextPhase(NAME, this::getNextPhase); } /** @@ -870,7 +864,7 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, } if (results.hasResult(shardIndex)) { - assert done.get() : "should only be called by subsequent phases, not during query"; + assert (int) OUTSTANDING_SHARDS.getAcquire(this) == 0 : "should only be called by subsequent phases, not during query"; assert failure == null : "shard failed before but shouldn't: " + failure; successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter } From 7894bc31eecfd11640d11cb6ac641622feed0d47 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 22 Jan 2025 00:42:46 +0100 Subject: [PATCH 098/132] fixes --- .../rest-api-spec/test/search/120_batch_reduce_size.yml | 2 +- .../action/search/SearchProgressActionListenerIT.java | 1 - .../elasticsearch/action/search/QueryPhaseResultConsumer.java | 2 +- .../elasticsearch/action/search/SearchPhaseControllerTests.java | 1 - 4 files changed, 2 insertions(+), 4 deletions(-) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml index ad8b5634b473d..c36d2580ec22e 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml @@ -48,7 +48,7 @@ setup: batched_reduce_size: 2 body: { "size" : 0, "aggs" : { "str_terms" : { "terms" : { "field" : "str" } } } } - - match: { num_reduce_phases: 4 } + - match: { num_reduce_phases: 3 } - match: { hits.total: 3 } - length: { aggregations.str_terms.buckets: 2 } - match: { aggregations.str_terms.buckets.0.key: "abc" } diff --git a/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java b/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java index 30291eb07f155..0e805ae996eda 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/action/search/SearchProgressActionListenerIT.java @@ -184,7 +184,6 @@ public SearchTask createTask(long id, String type, String action, TaskId parentT assertThat(numFetchResults.get(), equalTo(0)); assertThat(numFetchFailures.get(), equalTo(0)); } - assertThat(numReduces.get(), equalTo(searchResponse.get().getNumReducePhases())); } private static List createRandomIndices(Client client) { diff --git a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java index fc6bbedd083b4..42860290e9019 100644 --- a/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/QueryPhaseResultConsumer.java @@ -259,7 +259,7 @@ public SearchPhaseController.ReducedQueryPhase reduce() throws Exception { aggsList, topDocsList == null ? Collections.emptyList() : topDocsList, topDocsStats, - numReducePhases, + 2, false, aggReduceContextBuilder, queryPhaseRankCoordinatorContext, diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java index bf81486087361..fec305ced4fb7 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchPhaseControllerTests.java @@ -713,7 +713,6 @@ private void consumerTestCase(int numEmptyResponses) throws Exception { } SearchPhaseController.ReducedQueryPhase reduce = consumer.reduce(); - assertEquals(numTotalReducePhases, reduce.numReducePhases()); assertEquals(numTotalReducePhases, reductions.size()); assertAggReduction(request); Max max = (Max) reduce.aggregations().asList().get(0); From bd5c03bc90d8859c631a45eee5a4c1e2a1bfae40 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 22 Jan 2025 11:11:59 +0100 Subject: [PATCH 099/132] Simplify counting in AbstractSearchAsyncAction No need to do this so complicated, just count down one when we're actually done with a specific shard id. --- .../search/AbstractSearchAsyncAction.java | 73 +++++++------------ .../action/search/ExpandSearchPhaseTests.java | 1 - 2 files changed, 28 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index aeea0a5d65c8a..249e6a2b36fc2 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -42,6 +42,8 @@ import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.transport.Transport; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -90,15 +92,25 @@ abstract class AbstractSearchAsyncAction exten private final Object shardFailuresMutex = new Object(); private final AtomicBoolean hasShardResponse = new AtomicBoolean(false); private final AtomicInteger successfulOps = new AtomicInteger(); - private final AtomicInteger skippedOps = new AtomicInteger(); private final SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; protected final GroupShardsIterator toSkipShardsIts; protected final GroupShardsIterator shardsIts; private final SearchShardIterator[] shardIterators; - private final int expectedTotalOps; - private final AtomicInteger totalOps = new AtomicInteger(); + + private static final VarHandle OUTSTANDING_SHARDS; + + static { + try { + OUTSTANDING_SHARDS = MethodHandles.lookup().findVarHandle(AbstractSearchAsyncAction.class, "outstandingShards", int.class); + } catch (Exception e) { + throw new ExceptionInInitializerError(e); + } + } + + @SuppressWarnings("unused") // only accessed via #OUTSTANDING_SHARDS + private int outstandingShards; private final int maxConcurrentRequestsPerNode; private final Map pendingExecutionsPerNode = new ConcurrentHashMap<>(); private final boolean throttleConcurrentRequests; @@ -139,18 +151,12 @@ abstract class AbstractSearchAsyncAction exten } this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); this.shardsIts = new GroupShardsIterator<>(iterators); - + OUTSTANDING_SHARDS.setRelease(this, shardsIts.size()); this.shardIterators = iterators.toArray(new SearchShardIterator[0]); // we later compute the shard index based on the natural order of the shards // that participate in the search request. This means that this number is // consistent between two requests that target the same shards. Arrays.sort(shardIterators); - - // we need to add 1 for non active partition, since we count it in the total. This means for each shard in the iterator we sum up - // it's number of active shards but use 1 as the default if no replica of a shard is active at this point. - // on a per shards level we use shardIt.remaining() to increment the totalOps pointer but add 1 for the current shard result - // we process hence we add one for the non active partition here. - this.expectedTotalOps = shardsIts.totalSizeWith1ForEmpty(); this.maxConcurrentRequestsPerNode = maxConcurrentRequestsPerNode; // in the case were we have less shards than maxConcurrentRequestsPerNode we don't need to throttle this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size(); @@ -251,9 +257,8 @@ protected final void run() { void skipShard(SearchShardIterator iterator) { successfulOps.incrementAndGet(); - skippedOps.incrementAndGet(); assert iterator.skip(); - successfulShardExecution(iterator); + successfulShardExecution(); } private static boolean assertExecuteOnStartThread() { @@ -380,7 +385,7 @@ protected void executeNextPhase(String currentPhase, Supplier nextP "Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})", discrepancy, successfulOps.get(), - skippedOps.get(), + toSkipShardsIts.size(), getNumShards(), currentPhase ); @@ -449,17 +454,11 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } onShardGroupFailure(shardIndex, shard, e); } - final int totalOps = this.totalOps.incrementAndGet(); - if (totalOps == expectedTotalOps) { - onPhaseDone(); - } else if (totalOps > expectedTotalOps) { - throw new AssertionError( - "unexpected higher total ops [" + totalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) - ); + if (lastShard == false) { + performPhaseOnShard(shardIndex, shardIt, nextShard); } else { - if (lastShard == false) { - performPhaseOnShard(shardIndex, shardIt, nextShard); + if ((int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1) { + onPhaseDone(); } } } @@ -535,10 +534,10 @@ protected void onShardResult(Result result, SearchShardIterator shardIt) { if (logger.isTraceEnabled()) { logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); } - results.consumeResult(result, () -> onShardResultConsumed(result, shardIt)); + results.consumeResult(result, () -> onShardResultConsumed(result)); } - private void onShardResultConsumed(Result result, SearchShardIterator shardIt) { + private void onShardResultConsumed(Result result) { successfulOps.incrementAndGet(); // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level // so its ok concurrency wise to miss potentially the shard failures being created because of another failure @@ -552,28 +551,12 @@ private void onShardResultConsumed(Result result, SearchShardIterator shardIt) { // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc. // increment all the "future" shards to update the total ops since we some may work and some may not... // and when that happens, we break on total ops, so we must maintain them - successfulShardExecution(shardIt); + successfulShardExecution(); } - private void successfulShardExecution(SearchShardIterator shardsIt) { - final int remainingOpsOnIterator; - if (shardsIt.skip()) { - // It's possible that we're skipping a shard that's unavailable - // but its range was available in the IndexMetadata, in that - // case the shardsIt.remaining() would be 0, expectedTotalOps - // accounts for unavailable shards too. - remainingOpsOnIterator = Math.max(shardsIt.remaining(), 1); - } else { - remainingOpsOnIterator = shardsIt.remaining() + 1; - } - final int xTotalOps = totalOps.addAndGet(remainingOpsOnIterator); - if (xTotalOps == expectedTotalOps) { + private void successfulShardExecution() { + if ((int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1) { onPhaseDone(); - } else if (xTotalOps > expectedTotalOps) { - throw new AssertionError( - "unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures()) - ); } } @@ -640,7 +623,7 @@ private SearchResponse buildSearchResponse( scrollId, getNumShards(), numSuccess, - skippedOps.get(), + toSkipShardsIts.size(), buildTookInMillis(), failures, clusters, diff --git a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java index 65fdec96c92f0..5f84739599514 100644 --- a/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/ExpandSearchPhaseTests.java @@ -144,7 +144,6 @@ protected void run() { hits.decRef(); } } finally { - mockSearchPhaseContext.execute(() -> {}); var resp = mockSearchPhaseContext.searchResponse.get(); if (resp != null) { resp.decRef(); From 91c083a74fc51af57ef2e250dc292776445448b5 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 22 Jan 2025 13:40:42 +0100 Subject: [PATCH 100/132] fixes --- .../action/search/SearchQueryThenFetchAsyncAction.java | 8 ++++++-- .../search/SearchQueryThenFetchAsyncActionTests.java | 7 ++++++- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index cde1f2c8aa833..8a3467c2f4e52 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -605,6 +605,7 @@ private void run() { final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); doCheckNoMissingShards(NAME, request, shardsIts, SearchPhase::makeMissingShardsError); + final String localNodeId = searchTransportService.transportService().getLocalNode().getId(); final String localClusterAlias = request.getLocalClusterAlias(); for (int i = 0; i < shardsIts.size(); i++) { final SearchShardIterator shardRoutings = shardsIts.get(i); @@ -616,9 +617,12 @@ private void run() { failOnUnavailable(shardIndex, shardRoutings); } else { String clusterAlias = routing.getClusterAlias(); - if (supportsBatchedQuery && (clusterAlias == null || Objects.equals(localClusterAlias, clusterAlias))) { + final String nodeId = routing.getNodeId(); + if (supportsBatchedQuery + && localNodeId.equals(nodeId) == false // local requests don't need batching as there's no network latency + && (clusterAlias == null || Objects.equals(localClusterAlias, clusterAlias))) { perNodeQueries.computeIfAbsent( - routing.getNodeId(), + nodeId, ignored -> new NodeQueryRequest( new ArrayList<>(), request, diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 01ea257270178..96b4600d3a7b0 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -40,6 +40,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalAggregationTestCase; import org.elasticsearch.transport.Transport; +import org.elasticsearch.transport.TransportService; import java.util.Collections; import java.util.Map; @@ -51,6 +52,8 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class SearchQueryThenFetchAsyncActionTests extends ESTestCase { public void testBottomFieldSort() throws Exception { @@ -83,7 +86,9 @@ private void testCase(boolean withScroll, boolean withCollapse) throws Exception AtomicInteger numWithTopDocs = new AtomicInteger(); AtomicInteger successfulOps = new AtomicInteger(); AtomicBoolean canReturnNullResponse = new AtomicBoolean(false); - SearchTransportService searchTransportService = new SearchTransportService(null, null, null) { + var transportService = mock(TransportService.class); + when(transportService.getLocalNode()).thenReturn(primaryNode); + SearchTransportService searchTransportService = new SearchTransportService(transportService, null, null) { @Override public void sendExecuteQuery( Transport.Connection connection, From a5f74baa94dedaf1e0a228c7d702345bef84d19d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 22 Jan 2025 18:34:56 +0100 Subject: [PATCH 101/132] cleanup --- .../search/AbstractSearchAsyncAction.java | 42 ++++++++++++++++++ .../search/CanMatchPreFilterSearchPhase.java | 7 ++- .../action/search/SearchPhase.java | 43 ------------------- .../SearchQueryThenFetchAsyncAction.java | 3 +- 4 files changed, 49 insertions(+), 46 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index e9a6ce96785de..9c0f4a521ec71 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -54,6 +54,7 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Consumer; +import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -188,6 +189,47 @@ protected void notifyListShards( ); } + protected String missingShardsErrorMessage(StringBuilder missingShards) { + return makeMissingShardsError(missingShards); + } + + protected static String makeMissingShardsError(StringBuilder missingShards) { + return "Search rejected due to missing shards [" + + missingShards + + "]. Consider using `allow_partial_search_results` setting to bypass this error."; + } + + protected void doCheckNoMissingShards(String phaseName, SearchRequest request, GroupShardsIterator shardsIts) { + doCheckNoMissingShards(phaseName, request, shardsIts, this::missingShardsErrorMessage); + } + + protected static void doCheckNoMissingShards( + String phaseName, + SearchRequest request, + GroupShardsIterator shardsIts, + Function makeErrorMessage + ) { + assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; + if (request.allowPartialSearchResults() == false) { + final StringBuilder missingShards = new StringBuilder(); + // Fail-fast verification of all shards being available + for (int index = 0; index < shardsIts.size(); index++) { + final SearchShardIterator shardRoutings = shardsIts.get(index); + if (shardRoutings.size() == 0) { + if (missingShards.isEmpty() == false) { + missingShards.append(", "); + } + missingShards.append(shardRoutings.shardId()); + } + } + if (missingShards.isEmpty() == false) { + // Status red - shard is missing all copies and would produce partial results for an index search + final String msg = makeErrorMessage.apply(missingShards); + throw new SearchPhaseExecutionException(phaseName, msg, null, ShardSearchFailure.EMPTY_ARRAY); + } + } + } + /** * Registers a {@link Releasable} that will be closed when the search request finishes or fails. */ diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index dd2258eebf575..80444da37d827 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -208,7 +208,12 @@ private void runCoordinatorRewritePhase() { } else { GroupShardsIterator matchingShards = new GroupShardsIterator<>(matchedShardLevelRequests); // verify missing shards only for the shards that we hit for the query - SearchPhase.doCheckNoMissingShards("can_match", request, matchingShards, SearchPhase::makeMissingShardsError); + AbstractSearchAsyncAction.doCheckNoMissingShards( + "can_match", + request, + matchingShards, + AbstractSearchAsyncAction::makeMissingShardsError + ); new Round(matchingShards).run(); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index b3b0d535fd153..2cb0ea5389fb9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -10,14 +10,12 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.query.QuerySearchResult; import org.elasticsearch.transport.Transport; import java.util.Objects; -import java.util.function.Function; /** * Base class for all individual search phases like collecting distributed frequencies, fetching documents, querying shards. @@ -40,47 +38,6 @@ public String getName() { return name; } - protected String missingShardsErrorMessage(StringBuilder missingShards) { - return makeMissingShardsError(missingShards); - } - - protected static String makeMissingShardsError(StringBuilder missingShards) { - return "Search rejected due to missing shards [" - + missingShards - + "]. Consider using `allow_partial_search_results` setting to bypass this error."; - } - - protected void doCheckNoMissingShards(String phaseName, SearchRequest request, GroupShardsIterator shardsIts) { - doCheckNoMissingShards(phaseName, request, shardsIts, this::missingShardsErrorMessage); - } - - protected static void doCheckNoMissingShards( - String phaseName, - SearchRequest request, - GroupShardsIterator shardsIts, - Function makeErrorMessage - ) { - assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults"; - if (request.allowPartialSearchResults() == false) { - final StringBuilder missingShards = new StringBuilder(); - // Fail-fast verification of all shards being available - for (int index = 0; index < shardsIts.size(); index++) { - final SearchShardIterator shardRoutings = shardsIts.get(index); - if (shardRoutings.size() == 0) { - if (missingShards.isEmpty() == false) { - missingShards.append(", "); - } - missingShards.append(shardRoutings.shardId()); - } - } - if (missingShards.isEmpty() == false) { - // Status red - shard is missing all copies and would produce partial results for an index search - final String msg = makeErrorMessage.apply(missingShards); - throw new SearchPhaseExecutionException(phaseName, msg, null, ShardSearchFailure.EMPTY_ARRAY); - } - } - } - /** * Releases shard targets that are not used in the docsIdsToLoad. */ diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 8a3467c2f4e52..a6a983da3ec0c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -92,7 +92,6 @@ import static org.elasticsearch.action.search.AbstractSearchAsyncAction.DEFAULT_INDEX_BOOST; import static org.elasticsearch.action.search.AsyncSearchContext.buildShardFailures; -import static org.elasticsearch.action.search.SearchPhase.doCheckNoMissingShards; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; import static org.elasticsearch.core.Strings.format; @@ -604,7 +603,7 @@ private void run() { } final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); - doCheckNoMissingShards(NAME, request, shardsIts, SearchPhase::makeMissingShardsError); + AbstractSearchAsyncAction.doCheckNoMissingShards(NAME, request, shardsIts, AbstractSearchAsyncAction::makeMissingShardsError); final String localNodeId = searchTransportService.transportService().getLocalNode().getId(); final String localClusterAlias = request.getLocalClusterAlias(); for (int i = 0; i < shardsIts.size(); i++) { From e36a403a7f696c26347ec5e361419908b462a271 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 00:38:06 +0100 Subject: [PATCH 102/132] extract --- .../search/AbstractSearchAsyncAction.java | 52 ++++++------------- .../TransportOpenPointInTimeAction.java | 2 +- 2 files changed, 17 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 9c0f4a521ec71..c1aada94088f6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -69,7 +69,7 @@ * The fan out and collect algorithm is traditionally used as the initial phase which can either be a query execution or collection of * distributed frequencies */ -abstract class AbstractSearchAsyncAction extends SearchPhase implements AsyncSearchContext { +abstract class AbstractSearchAsyncAction implements AsyncSearchContext { static final float DEFAULT_INDEX_BOOST = 1.0f; private final Logger logger; private final NamedWriteableRegistry namedWriteableRegistry; @@ -108,6 +108,7 @@ abstract class AbstractSearchAsyncAction exten // protected for tests protected final List releasables = new ArrayList<>(); + protected final String name; AbstractSearchAsyncAction( String name, @@ -128,7 +129,7 @@ abstract class AbstractSearchAsyncAction exten int maxConcurrentRequestsPerNode, SearchResponse.Clusters clusters ) { - super(name); + this.name = name; this.namedWriteableRegistry = namedWriteableRegistry; final List toSkipIterators = new ArrayList<>(); final List iterators = new ArrayList<>(); @@ -262,11 +263,17 @@ public final void start() { ); return; } - executePhase(this); + try { + run(); + } catch (RuntimeException e) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, name), e); + } + onPhaseFailure(name, "", e); + } } - @Override - protected final void run() { + private void run() { for (final SearchShardIterator iterator : toSkipShardsIts) { assert iterator.skip(); skipShard(iterator); @@ -276,7 +283,7 @@ protected final void run() { shardIndexMap.put(shardIterators[i], i); } if (shardsIts.size() > 0) { - doCheckNoMissingShards(getName(), request, shardsIts); + doCheckNoMissingShards(name, request, shardsIts); for (int i = 0; i < shardsIts.size(); i++) { final SearchShardIterator shardRoutings = shardsIts.get(i); assert shardRoutings.skip() == false; @@ -299,32 +306,6 @@ void skipShard(SearchShardIterator iterator) { successfulShardExecution(iterator); } - private static boolean assertExecuteOnStartThread() { - // Ensure that the current code has the following stacktrace: - // AbstractSearchAsyncAction#start -> AbstractSearchAsyncAction#executePhase -> AbstractSearchAsyncAction#performPhaseOnShard - final StackTraceElement[] stackTraceElements = Thread.currentThread().getStackTrace(); - assert stackTraceElements.length >= 6 : stackTraceElements; - int index = 0; - assert stackTraceElements[index++].getMethodName().equals("getStackTrace"); - assert stackTraceElements[index++].getMethodName().equals("assertExecuteOnStartThread"); - assert stackTraceElements[index++].getMethodName().equals("failOnUnavailable"); - if (stackTraceElements[index].getMethodName().equals("performPhaseOnShard")) { - assert stackTraceElements[index].getClassName().endsWith("CanMatchPreFilterSearchPhase"); - index++; - } - assert stackTraceElements[index].getClassName().endsWith("AbstractSearchAsyncAction"); - assert stackTraceElements[index++].getMethodName().equals("run"); - - assert stackTraceElements[index].getClassName().endsWith("AbstractSearchAsyncAction"); - assert stackTraceElements[index++].getMethodName().equals("executePhase"); - - assert stackTraceElements[index].getClassName().endsWith("AbstractSearchAsyncAction"); - assert stackTraceElements[index++].getMethodName().equals("start"); - - assert stackTraceElements[index].getClassName().endsWith("AbstractSearchAsyncAction") == false; - return true; - } - private void performPhaseOnShard(final int shardIndex, final SearchShardIterator shardIt, final SearchShardTarget shard) { if (throttleConcurrentRequests) { var pendingExecutions = pendingExecutionsPerNode.computeIfAbsent( @@ -366,7 +347,6 @@ public void onFailure(Exception e) { } private void failOnUnavailable(int shardIndex, SearchShardIterator shardIt) { - assert assertExecuteOnStartThread(); SearchShardTarget unassignedShard = new SearchShardTarget(null, shardIt.shardId(), shardIt.getClusterAlias()); onShardFailure(shardIndex, unassignedShard, shardIt, new NoShardAvailableActionException(shardIt.shardId())); } @@ -489,7 +469,7 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } else if (totalOps > expectedTotalOps) { throw new AssertionError( "unexpected higher total ops [" + totalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures(shardFailures)) + new SearchPhaseExecutionException(name, "Shard failures", null, buildShardFailures(shardFailures)) ); } else { if (lastShard == false) { @@ -607,7 +587,7 @@ private void successfulShardExecution(SearchShardIterator shardsIt) { } else if (xTotalOps > expectedTotalOps) { throw new AssertionError( "unexpected higher total ops [" + xTotalOps + "] compared to expected [" + expectedTotalOps + "]", - new SearchPhaseExecutionException(getName(), "Shard failures", null, buildShardFailures(shardFailures)) + new SearchPhaseExecutionException(name, "Shard failures", null, buildShardFailures(shardFailures)) ); } } @@ -767,7 +747,7 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C * @see #onShardResult(SearchPhaseResult, SearchShardIterator) */ private void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() - executeNextPhase(this.getName(), this::getNextPhase); + executeNextPhase(name, this::getNextPhase); } /** diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java index 4516b7df3f45f..1f07ccfabbef6 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java @@ -268,7 +268,7 @@ protected void executePhaseOnShard( @Override protected SearchPhase getNextPhase() { - return new SearchPhase(getName()) { + return new SearchPhase(name) { @Override protected void run() { sendSearchResponse(SearchResponseSections.EMPTY_WITH_TOTAL_HITS, results.getAtomicArray()); From cde86fa043817557b3f0569c785f253180da8775 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 01:12:52 +0100 Subject: [PATCH 103/132] extract --- .../search/AbstractSearchAsyncAction.java | 89 ++------------ .../action/search/AsyncSearchContext.java | 113 +++++++++++++++--- .../action/search/CountedCollector.java | 4 +- .../action/search/ExpandSearchPhase.java | 4 +- .../action/search/FetchLookupFieldsPhase.java | 10 +- .../action/search/FetchSearchPhase.java | 18 +-- .../action/search/RankFeaturePhase.java | 8 +- .../action/search/SearchContextId.java | 2 +- .../action/search/SearchPhase.java | 4 +- .../SearchQueryThenFetchAsyncAction.java | 71 +++-------- .../action/search/TransportSearchAction.java | 2 +- .../action/search/FetchSearchPhaseTests.java | 2 +- .../action/search/MockSearchPhaseContext.java | 2 +- .../SearchQueryThenFetchAsyncActionTests.java | 2 +- 14 files changed, 153 insertions(+), 178 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index c1aada94088f6..d5682ce0ab54e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -28,12 +28,10 @@ import org.elasticsearch.common.util.Maps; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; -import org.elasticsearch.core.Releasables; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; @@ -58,7 +56,6 @@ import java.util.function.Supplier; import java.util.stream.Collectors; -import static org.elasticsearch.action.search.AsyncSearchContext.buildShardFailures; import static org.elasticsearch.core.Strings.format; /** @@ -69,21 +66,17 @@ * The fan out and collect algorithm is traditionally used as the initial phase which can either be a query execution or collection of * distributed frequencies */ -abstract class AbstractSearchAsyncAction implements AsyncSearchContext { +abstract class AbstractSearchAsyncAction extends AsyncSearchContext { static final float DEFAULT_INDEX_BOOST = 1.0f; private final Logger logger; - private final NamedWriteableRegistry namedWriteableRegistry; private final SearchTransportService searchTransportService; private final Executor executor; - private final ActionListener listener; - private final SearchRequest request; /** * Used by subclasses to resolve node ids to DiscoveryNodes. **/ private final BiFunction nodeIdToConnection; private final SearchTask task; - protected final SearchPhaseResults results; private final long clusterStateVersion; private final TransportVersion minTransportVersion; private final Map aliasFilter; @@ -107,7 +100,6 @@ abstract class AbstractSearchAsyncAction imple private final AtomicBoolean requestCancelled = new AtomicBoolean(); // protected for tests - protected final List releasables = new ArrayList<>(); protected final String name; AbstractSearchAsyncAction( @@ -129,8 +121,8 @@ abstract class AbstractSearchAsyncAction imple int maxConcurrentRequestsPerNode, SearchResponse.Clusters clusters ) { + super(request, resultConsumer, namedWriteableRegistry, listener); this.name = name; - this.namedWriteableRegistry = namedWriteableRegistry; final List toSkipIterators = new ArrayList<>(); final List iterators = new ArrayList<>(); for (final SearchShardIterator iterator : shardsIts) { @@ -161,18 +153,12 @@ abstract class AbstractSearchAsyncAction imple this.logger = logger; this.searchTransportService = searchTransportService; this.executor = executor; - this.request = request; this.task = task; - this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); this.nodeIdToConnection = nodeIdToConnection; this.concreteIndexBoosts = concreteIndexBoosts; this.clusterStateVersion = clusterState.version(); this.minTransportVersion = clusterState.getMinTransportVersion(); this.aliasFilter = aliasFilter; - this.results = resultConsumer; - // register the release of the query consumer to free up the circuit breaker memory - // at the end of the search - addReleasable(resultConsumer); this.clusters = clusters; } @@ -231,13 +217,6 @@ protected static void doCheckNoMissingShards( } } - /** - * Registers a {@link Releasable} that will be closed when the search request finishes or fails. - */ - public void addReleasable(Releasable releasable) { - releasables.add(releasable); - } - /** * Builds how long it took to execute the search. */ @@ -426,18 +405,14 @@ public void executeNextPhase(String currentPhase, Supplier nextPhas clusterStateVersion ); } - executePhase(nextPhase); - } - } - - private void executePhase(SearchPhase phase) { - try { - phase.run(); - } catch (RuntimeException e) { - if (logger.isDebugEnabled()) { - logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, phase.getName()), e); + try { + nextPhase.run(); + } catch (RuntimeException e) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, nextPhase.getName()), e); + } + onPhaseFailure(nextPhase.getName(), "", e); } - onPhaseFailure(phase.getName(), "", e); } } @@ -613,27 +588,6 @@ public OriginalIndices getOriginalIndices(int shardIndex) { return shardIterators[shardIndex].getOriginalIndices(); } - /** - * Checks if the given context id is part of the point in time of this search (if exists). - * We should not release search contexts that belong to the point in time during or after searches. - */ - public boolean isPartOfPointInTime(ShardSearchContextId contextId) { - return isPartOfPIT(namedWriteableRegistry, request, contextId); - } - - public static boolean isPartOfPIT( - NamedWriteableRegistry namedWriteableRegistry, - SearchRequest request, - ShardSearchContextId contextId - ) { - final PointInTimeBuilder pointInTimeBuilder = request.pointInTimeBuilder(); - if (pointInTimeBuilder != null) { - return request.pointInTimeBuilder().getSearchContextId(namedWriteableRegistry).contains(contextId); - } else { - return false; - } - } - private SearchResponse buildSearchResponse( SearchResponseSections internalSearchResponse, ShardSearchFailure[] failures, @@ -668,7 +622,7 @@ boolean buildPointInTimeFromSearchResults() { * @param internalSearchResponse the internal search response * @param queryResults the results of the query phase */ - public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { + public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { ShardSearchFailure[] failures = buildShardFailures(shardFailures); Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; @@ -704,29 +658,6 @@ public void onPhaseFailure(String phase, String msg, Throwable cause) { raisePhaseFailure(new SearchPhaseExecutionException(phase, msg, cause, buildShardFailures(shardFailures))); } - /** - * This method should be called if a search phase failed to ensure all relevant reader contexts are released. - * This method will also notify the listener and sends back a failure to the user. - * - * @param exception the exception explaining or causing the phase failure - */ - private void raisePhaseFailure(SearchPhaseExecutionException exception) { - results.getSuccessfulResults().forEach((entry) -> { - // Do not release search contexts that are part of the point in time - if (entry.getContextId() != null && isPartOfPointInTime(entry.getContextId()) == false) { - try { - SearchShardTarget searchShardTarget = entry.getSearchShardTarget(); - Transport.Connection connection = getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId()); - sendReleaseSearchContext(entry.getContextId(), connection); - } catch (Exception inner) { - inner.addSuppressed(exception); - logger.trace("failed to release context", inner); - } - } - }); - listener.onFailure(exception); - } - /** * Releases a search context with the given context ID on the node the given connection is connected to. * @see org.elasticsearch.search.query.QuerySearchResult#getContextId() diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 9a56ed39f6a8c..033246aa10569 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -9,19 +9,55 @@ package org.elasticsearch.action.search; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; +import org.elasticsearch.core.Releasables; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; +import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.transport.Transport; +import java.util.ArrayList; import java.util.List; import java.util.function.Supplier; -public interface AsyncSearchContext { +public abstract class AsyncSearchContext { + + private static final Logger logger = LogManager.getLogger(AsyncSearchContext.class); + + protected final SearchRequest request; + + protected final SearchPhaseResults results; + + private final NamedWriteableRegistry namedWriteableRegistry; + + protected final ActionListener listener; + + // protected for tests + protected final List releasables = new ArrayList<>(); + + protected AsyncSearchContext( + SearchRequest request, + SearchPhaseResults results, + NamedWriteableRegistry namedWriteableRegistry, + ActionListener listener + ) { + this.request = request; + this.results = results; + this.namedWriteableRegistry = namedWriteableRegistry; + this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); + ; + // register the release of the query consumer to free up the circuit breaker memory + // at the end of the search + addReleasable(results); + } static ShardSearchFailure[] buildShardFailures(SetOnce> shardFailuresRef) { AtomicArray shardFailures = shardFailuresRef.get(); @@ -36,27 +72,72 @@ static ShardSearchFailure[] buildShardFailures(SetOnce queryResults); - - SearchTransportService getSearchTransport(); - - SearchTask getTask(); + static boolean isPartOfPIT(NamedWriteableRegistry namedWriteableRegistry, SearchRequest request, ShardSearchContextId contextId) { + final PointInTimeBuilder pointInTimeBuilder = request.pointInTimeBuilder(); + if (pointInTimeBuilder != null) { + return request.pointInTimeBuilder().getSearchContextId(namedWriteableRegistry).contains(contextId); + } else { + return false; + } + } - void onPhaseFailure(String phase, String msg, Throwable cause); + abstract SearchRequest getRequest(); - void addReleasable(Releasable releasable); + abstract void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults); - void execute(Runnable command); + abstract SearchTransportService getSearchTransport(); - void onShardFailure(int shardIndex, SearchShardTarget shard, Exception e); + abstract SearchTask getTask(); - Transport.Connection getConnection(String clusterAlias, String nodeId); + abstract void onPhaseFailure(String phase, String msg, Throwable cause); - OriginalIndices getOriginalIndices(int shardIndex); + /** + * Registers a {@link Releasable} that will be closed when the search request finishes or fails. + */ + public final void addReleasable(Releasable releasable) { + releasables.add(releasable); + } - void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection); + abstract void execute(Runnable command); + + abstract void onShardFailure(int shardIndex, SearchShardTarget shard, Exception e); + + abstract Transport.Connection getConnection(String clusterAlias, String nodeId); + + abstract OriginalIndices getOriginalIndices(int shardIndex); + + abstract void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection); + + abstract void executeNextPhase(String currentPhase, Supplier nextPhaseSupplier); + + /** + * This method should be called if a search phase failed to ensure all relevant reader contexts are released. + * This method will also notify the listener and sends back a failure to the user. + * + * @param exception the exception explaining or causing the phase failure + */ + protected final void raisePhaseFailure(SearchPhaseExecutionException exception) { + results.getSuccessfulResults().forEach((entry) -> { + // Do not release search contexts that are part of the point in time + if (entry.getContextId() != null && isPartOfPointInTime(entry.getContextId()) == false) { + try { + SearchShardTarget searchShardTarget = entry.getSearchShardTarget(); + Transport.Connection connection = getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId()); + sendReleaseSearchContext(entry.getContextId(), connection); + } catch (Exception inner) { + inner.addSuppressed(exception); + logger.trace("failed to release context", inner); + } + } + }); + listener.onFailure(exception); + } - void executeNextPhase(String currentPhase, Supplier nextPhaseSupplier); + /** + * Checks if the given context id is part of the point in time of this search (if exists). + * We should not release search contexts that belong to the point in time during or after searches. + */ + public boolean isPartOfPointInTime(ShardSearchContextId contextId) { + return isPartOfPIT(namedWriteableRegistry, request, contextId); + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java b/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java index 45b8b1804f735..6eed47fd7f2cf 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java +++ b/server/src/main/java/org/elasticsearch/action/search/CountedCollector.java @@ -22,9 +22,9 @@ final class CountedCollector { private final SearchPhaseResults resultConsumer; private final CountDown counter; private final Runnable onFinish; - private final AsyncSearchContext context; + private final AsyncSearchContext context; - CountedCollector(SearchPhaseResults resultConsumer, int expectedOps, Runnable onFinish, AsyncSearchContext context) { + CountedCollector(SearchPhaseResults resultConsumer, int expectedOps, Runnable onFinish, AsyncSearchContext context) { this.resultConsumer = resultConsumer; this.counter = new CountDown(expectedOps); this.onFinish = onFinish; diff --git a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java index f31357170aa05..f513242e63d83 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/ExpandSearchPhase.java @@ -34,11 +34,11 @@ final class ExpandSearchPhase extends SearchPhase { static final String NAME = "expand"; - private final AsyncSearchContext context; + private final AsyncSearchContext context; private final SearchHits searchHits; private final Supplier nextPhase; - ExpandSearchPhase(AsyncSearchContext context, SearchHits searchHits, Supplier nextPhase) { + ExpandSearchPhase(AsyncSearchContext context, SearchHits searchHits, Supplier nextPhase) { super(NAME); this.context = context; this.searchHits = searchHits; diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java index 90fa6be97dc44..cb51aeb6642bc 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchLookupFieldsPhase.java @@ -36,11 +36,15 @@ final class FetchLookupFieldsPhase extends SearchPhase { static final String NAME = "fetch_lookup_fields"; - private final AsyncSearchContext context; + private final AsyncSearchContext context; private final SearchResponseSections searchResponse; - private final AtomicArray queryResults; + private final AtomicArray queryResults; - FetchLookupFieldsPhase(AsyncSearchContext context, SearchResponseSections searchResponse, AtomicArray queryResults) { + FetchLookupFieldsPhase( + AsyncSearchContext context, + SearchResponseSections searchResponse, + AtomicArray queryResults + ) { super(NAME); this.context = context; this.searchResponse = searchResponse; diff --git a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java index effc8ab5099cf..26388196a8838 100644 --- a/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/FetchSearchPhase.java @@ -39,20 +39,20 @@ final class FetchSearchPhase extends SearchPhase { static final String NAME = "fetch"; - private final AtomicArray searchPhaseShardResults; - private final BiFunction, SearchPhase> nextPhaseFactory; - private final AsyncSearchContext context; + private final AtomicArray searchPhaseShardResults; + private final BiFunction, SearchPhase> nextPhaseFactory; + private final AsyncSearchContext context; private final SearchProgressListener progressListener; private final AggregatedDfs aggregatedDfs; @Nullable - private final SearchPhaseResults resultConsumer; + private final SearchPhaseResults resultConsumer; private final SearchPhaseController.ReducedQueryPhase reducedQueryPhase; private final int numShards; FetchSearchPhase( - SearchPhaseResults resultConsumer, + SearchPhaseResults resultConsumer, AggregatedDfs aggregatedDfs, - AsyncSearchContext context, + AsyncSearchContext context, @Nullable SearchPhaseController.ReducedQueryPhase reducedQueryPhase ) { this( @@ -69,11 +69,11 @@ final class FetchSearchPhase extends SearchPhase { } FetchSearchPhase( - SearchPhaseResults resultConsumer, + SearchPhaseResults resultConsumer, AggregatedDfs aggregatedDfs, - AsyncSearchContext context, + AsyncSearchContext context, @Nullable SearchPhaseController.ReducedQueryPhase reducedQueryPhase, - BiFunction, SearchPhase> nextPhaseFactory + BiFunction, SearchPhase> nextPhaseFactory ) { super(NAME); this.searchPhaseShardResults = resultConsumer.getAtomicArray(); diff --git a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java index 0bf2a42d28d5a..1aacf20e6e587 100644 --- a/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/RankFeaturePhase.java @@ -40,17 +40,17 @@ public class RankFeaturePhase extends SearchPhase { static final String NAME = "rank-feature"; private static final Logger logger = LogManager.getLogger(RankFeaturePhase.class); - private final AsyncSearchContext context; - final SearchPhaseResults queryPhaseResults; + private final AsyncSearchContext context; + final SearchPhaseResults queryPhaseResults; final SearchPhaseResults rankPhaseResults; private final AggregatedDfs aggregatedDfs; private final SearchProgressListener progressListener; private final RankFeaturePhaseRankCoordinatorContext rankFeaturePhaseRankCoordinatorContext; RankFeaturePhase( - SearchPhaseResults queryPhaseResults, + SearchPhaseResults queryPhaseResults, AggregatedDfs aggregatedDfs, - AsyncSearchContext context, + AsyncSearchContext context, RankFeaturePhaseRankCoordinatorContext rankFeaturePhaseRankCoordinatorContext ) { super(NAME); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchContextId.java b/server/src/main/java/org/elasticsearch/action/search/SearchContextId.java index c2f1510341fb0..54e3ba205dccb 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchContextId.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchContextId.java @@ -58,7 +58,7 @@ public boolean contains(ShardSearchContextId contextId) { } public static BytesReference encode( - List searchPhaseResults, + List searchPhaseResults, Map aliasFilter, TransportVersion version, ShardSearchFailure[] shardFailures diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java index 2cb0ea5389fb9..4a8c6906f7db7 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhase.java @@ -41,7 +41,7 @@ public String getName() { /** * Releases shard targets that are not used in the docsIdsToLoad. */ - protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPhaseResult, AsyncSearchContext context) { + protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPhaseResult, AsyncSearchContext context) { // we only release search context that we did not fetch from, if we are not scrolling // or using a PIT and if it has at least one hit that didn't make it to the global topDocs // phaseResult.getContextId() is the same for query & rank feature results @@ -51,7 +51,7 @@ protected static void releaseIrrelevantSearchContext(SearchPhaseResult searchPha if (phaseResult != null && (phaseResult.hasSearchContext() || (phaseResult instanceof QuerySearchResult q && q.isReduced() && q.getContextId() != null)) && context.getRequest().scroll() == null - && (AbstractSearchAsyncAction.isPartOfPIT(null, context.getRequest(), phaseResult.getContextId()) == false)) { + && (AsyncSearchContext.isPartOfPIT(null, context.getRequest(), phaseResult.getContextId()) == false)) { try { logger.trace("trying to release search context [{}]", phaseResult.getContextId()); SearchShardTarget shardTarget = phaseResult.getSearchShardTarget(); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index a6a983da3ec0c..c83c8de12c9ed 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -43,8 +43,6 @@ import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.core.RefCounted; -import org.elasticsearch.core.Releasable; -import org.elasticsearch.core.Releasables; import org.elasticsearch.core.SimpleRefCounted; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.shard.ShardId; @@ -91,28 +89,23 @@ import java.util.stream.Collectors; import static org.elasticsearch.action.search.AbstractSearchAsyncAction.DEFAULT_INDEX_BOOST; -import static org.elasticsearch.action.search.AsyncSearchContext.buildShardFailures; import static org.elasticsearch.action.search.SearchPhaseController.getTopDocsSize; import static org.elasticsearch.core.Strings.format; -public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { +public class SearchQueryThenFetchAsyncAction extends AsyncSearchContext { private static final String NAME = "query"; private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class); - private final NamedWriteableRegistry namedWriteableRegistry; private final SearchTransportService searchTransportService; private final Executor executor; - private final ActionListener listener; - private final SearchRequest request; /** * Used by subclasses to resolve node ids to DiscoveryNodes. **/ private final BiFunction nodeIdToConnection; private final SearchTask task; - protected final SearchPhaseResults results; private final TransportVersion minNodeVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; @@ -141,9 +134,6 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { private int outstandingShards; - // protected for tests - protected final List releasables = new ArrayList<>(); - private final SearchProgressListener progressListener; // informations to track the best bottom top doc globally. @@ -159,7 +149,7 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { Map aliasFilter, Map concreteIndexBoosts, Executor executor, - SearchPhaseResults resultConsumer, + SearchPhaseResults resultConsumer, SearchRequest request, ActionListener listener, GroupShardsIterator shardsIts, @@ -169,7 +159,7 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { SearchResponse.Clusters clusters, Client client ) { - this.namedWriteableRegistry = namedWriteableRegistry; + super(request, resultConsumer, namedWriteableRegistry, listener); final List toSkipIterators = new ArrayList<>(); final List iterators = new ArrayList<>(); for (final SearchShardIterator iterator : shardsIts) { @@ -192,14 +182,11 @@ public class SearchQueryThenFetchAsyncAction implements AsyncSearchContext { this.timeProvider = timeProvider; this.searchTransportService = searchTransportService; this.executor = executor; - this.request = request; this.task = task; - this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); this.nodeIdToConnection = nodeIdToConnection; this.concreteIndexBoosts = concreteIndexBoosts; this.minNodeVersion = clusterState.getMinTransportVersion(); this.aliasFilter = aliasFilter; - this.results = resultConsumer; // register the release of the query consumer to free up the circuit breaker memory // at the end of the search releasables.add(resultConsumer); @@ -255,7 +242,7 @@ public SearchRequest getRequest() { * @param internalSearchResponse the internal search response * @param queryResults the results of the query phase */ - public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { + public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { ShardSearchFailure[] failures = buildShardFailures(shardFailures); Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; @@ -519,7 +506,7 @@ protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget progressListener.notifyQueryFailure(shardIndex, shardTarget, exc); } - protected void onShardResult(SearchPhaseResult result) { + protected void onShardResult(Result result) { QuerySearchResult queryResult = result.queryResult(); if (queryResult.isNull() == false // disable sort optims for scroll requests because they keep track of the last bottom doc locally (per shard) @@ -549,8 +536,8 @@ protected void onShardResult(SearchPhaseResult result) { static SearchPhase nextPhase( Client client, - AsyncSearchContext context, - SearchPhaseResults queryResults, + AsyncSearchContext context, + SearchPhaseResults queryResults, AggregatedDfs aggregatedDfs ) { var rankFeaturePhaseCoordCtx = RankFeaturePhase.coordinatorContext(context.getRequest().source(), client); @@ -683,7 +670,9 @@ public void handleResponse(NodeQueryResponse response) { case SearchPhaseResult q -> { q.setShardIndex(shardIdx); q.setSearchShardTarget(target); - onShardResult(q); + @SuppressWarnings("unchecked") + var res = (Result) q; + onShardResult(res); } case null, default -> { assert false : "impossible [" + response.results[i] + "]"; @@ -769,7 +758,9 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat @Override public void innerOnResponse(SearchPhaseResult result) { try { - onShardResult(result); + @SuppressWarnings("unchecked") + var res = (Result) result; + onShardResult(res); } catch (Exception exc) { // TODO: this looks like a nasty bug where it to actually happen assert false : exc; @@ -956,39 +947,11 @@ public void onPhaseFailure(String phase, String msg, Throwable cause) { raisePhaseFailure(new SearchPhaseExecutionException(phase, msg, cause, buildShardFailures(shardFailures))); } - @Override - public void addReleasable(Releasable releasable) { - releasables.add(releasable); - } - @Override public void execute(Runnable command) { executor.execute(command); } - /** - * This method should be called if a search phase failed to ensure all relevant reader contexts are released. - * This method will also notify the listener and sends back a failure to the user. - * - * @param exception the exception explaining or causing the phase failure - */ - private void raisePhaseFailure(SearchPhaseExecutionException exception) { - results.getSuccessfulResults().forEach((entry) -> { - // Do not release search contexts that are part of the point in time - if (entry.getContextId() != null && isPartOfPointInTime(entry.getContextId()) == false) { - try { - SearchShardTarget searchShardTarget = entry.getSearchShardTarget(); - Transport.Connection connection = getConnection(searchShardTarget.getClusterAlias(), searchShardTarget.getNodeId()); - sendReleaseSearchContext(entry.getContextId(), connection); - } catch (Exception inner) { - inner.addSuppressed(exception); - logger.trace("failed to release context", inner); - } - } - }); - listener.onFailure(exception); - } - @Override public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { assert isPartOfPointInTime(contextId) == false : "Must not release point in time context [" + contextId + "]"; @@ -997,10 +960,6 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C } } - public boolean isPartOfPointInTime(ShardSearchContextId contextId) { - return AbstractSearchAsyncAction.isPartOfPIT(namedWriteableRegistry, request, contextId); - } - private void onShardResultConsumed(SearchPhaseResult result) { successfulOps.incrementAndGet(); // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level @@ -1062,7 +1021,7 @@ private static void maybeRelease(SearchService searchService, NodeQueryRequest r if (phaseResult != null && phaseResult.hasSearchContext() && request.searchRequest.scroll() == null - && (AbstractSearchAsyncAction.isPartOfPIT(null, request.searchRequest, phaseResult.getContextId()) == false)) { + && (AsyncSearchContext.isPartOfPIT(null, request.searchRequest, phaseResult.getContextId()) == false)) { searchService.freeReaderContext(phaseResult.getContextId()); } } @@ -1267,7 +1226,7 @@ private QueryPerNodeState( && q.hasSuggestHits() == false && q.getRankShardResult() == null && searchRequest.searchRequest.scroll() == null - && (AbstractSearchAsyncAction.isPartOfPIT(null, searchRequest.searchRequest, q.getContextId()) == false)) { + && (AsyncSearchContext.isPartOfPIT(null, searchRequest.searchRequest, q.getContextId()) == false)) { if (dependencies.searchService.freeReaderContext(q.getContextId())) { q.clearContextId(); } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 8bee1d3d20c2a..dc82cdb1a4d65 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -1544,7 +1544,7 @@ public void runNewSearchPhase( searchPhase.start(); } else { assert searchRequest.searchType() == QUERY_THEN_FETCH : searchRequest.searchType(); - var searchPhase = new SearchQueryThenFetchAsyncAction( + var searchPhase = new SearchQueryThenFetchAsyncAction<>( namedWriteableRegistry, searchTransportService, connectionLookup, diff --git a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java index fd60621c7e400..e23b78522b13b 100644 --- a/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/FetchSearchPhaseTests.java @@ -759,7 +759,7 @@ public void sendExecuteFetch( } - private static BiFunction, SearchPhase> searchPhaseFactory( + private static BiFunction, SearchPhase> searchPhaseFactory( MockSearchPhaseContext mockSearchPhaseContext ) { return (searchResponse, scrollId) -> new SearchPhase("test") { diff --git a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java index e8e12300c23e3..5d6cf92bfcc7d 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java +++ b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java @@ -87,7 +87,7 @@ public OriginalIndices getOriginalIndices(int shardIndex) { } @Override - public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { + public void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults) { String scrollId = getRequest().scroll() != null ? TransportSearchHelper.buildScrollId(queryResults) : null; BytesReference searchContextId = getRequest().pointInTimeBuilder() != null ? new BytesArray(TransportSearchHelper.buildScrollId(queryResults)) diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java index 96b4600d3a7b0..2362544b14cbf 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncActionTests.java @@ -191,7 +191,7 @@ public void sendExecuteQuery( exc -> {} ) ) { - SearchQueryThenFetchAsyncAction action = new SearchQueryThenFetchAsyncAction( + SearchQueryThenFetchAsyncAction action = new SearchQueryThenFetchAsyncAction<>( null, searchTransportService, (clusterAlias, node) -> lookup.get(node), From d8beb195bf544f65ed2ef893b7ec8d7c9ce842c4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 01:23:22 +0100 Subject: [PATCH 104/132] extract --- .../action/search/AbstractSearchAsyncAction.java | 14 ++++---------- .../action/search/AsyncSearchContext.java | 9 ++++++++- .../search/SearchQueryThenFetchAsyncAction.java | 12 ++++-------- 3 files changed, 16 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index d5682ce0ab54e..eac02c941a340 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -83,7 +83,6 @@ abstract class AbstractSearchAsyncAction exten private final Map concreteIndexBoosts; private final SetOnce> shardFailures = new SetOnce<>(); private final Object shardFailuresMutex = new Object(); - private final AtomicBoolean hasShardResponse = new AtomicBoolean(false); private final AtomicInteger successfulOps = new AtomicInteger(); private final AtomicInteger skippedOps = new AtomicInteger(); private final SearchTimeProvider timeProvider; @@ -521,7 +520,9 @@ static boolean isTaskCancelledException(Exception e) { protected void onShardResult(Result result, SearchShardIterator shardIt) { assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; - hasShardResponse.set(true); + if (hasShardResponse == false) { + hasShardResponse = true; + } if (logger.isTraceEnabled()) { logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); } @@ -574,13 +575,6 @@ public final SearchTask getTask() { return task; } - /** - * Returns the currently executing search request - */ - public final SearchRequest getRequest() { - return request; - } - /** * Returns the targeted {@link OriginalIndices} for the provided {@code shardIndex}. */ @@ -729,7 +723,7 @@ protected final ShardSearchRequest buildShardSearchRequest(SearchShardIterator s // can return a null response if the request rewrites to match none rather // than creating an empty response in the search thread pool. // Note that, we have to disable this shortcut for queries that create a context (scroll and search context). - shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse.get() && shardRequest.scroll() == null); + shardRequest.canReturnNullResponseIfMatchNoDocs(hasShardResponse && shardRequest.scroll() == null); return shardRequest; } diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 033246aa10569..b39b01ef1a09b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -40,6 +40,8 @@ public abstract class AsyncSearchContext { protected final ActionListener listener; + protected volatile boolean hasShardResponse = false; + // protected for tests protected final List releasables = new ArrayList<>(); @@ -81,7 +83,12 @@ static boolean isPartOfPIT(NamedWriteableRegistry namedWriteableRegistry, Search } } - abstract SearchRequest getRequest(); + /** + * Returns the currently executing search request + */ + public final SearchRequest getRequest() { + return request; + } abstract void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index c83c8de12c9ed..d7df44c021f67 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -111,7 +111,6 @@ public class SearchQueryThenFetchAsyncAction e private final Map concreteIndexBoosts; private final SetOnce> shardFailures = new SetOnce<>(); private final Object shardFailuresMutex = new Object(); - private final AtomicBoolean hasShardResponse = new AtomicBoolean(false); private final AtomicInteger successfulOps; private final TransportSearchAction.SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; @@ -231,11 +230,6 @@ public final void start() { } } - @Override - public SearchRequest getRequest() { - return request; - } - /** * Builds and sends the final search response back to the user. * @@ -527,7 +521,9 @@ protected void onShardResult(Result result) { } assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; - hasShardResponse.set(true); + if (hasShardResponse == false) { + hasShardResponse = true; + } if (logger.isTraceEnabled()) { logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); } @@ -750,7 +746,7 @@ protected void performPhaseOnShard(final int shardIndex, final SearchShardIterat request, results.getNumShards(), timeProvider.absoluteStartMillis(), - hasShardResponse.get() + hasShardResponse ) ), task, From f7fe431dd104c4ed55155296e86a7d9ad343d900 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 14:17:58 +0100 Subject: [PATCH 105/132] extract --- .../search/AbstractSearchAsyncAction.java | 51 +++-------------- .../action/search/AsyncSearchContext.java | 56 ++++++++++++++++++- .../SearchQueryThenFetchAsyncAction.java | 39 +------------ .../action/search/MockSearchPhaseContext.java | 2 +- 4 files changed, 63 insertions(+), 85 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index cba9bfd8de1e6..9d8ba0d16c5b8 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -50,8 +50,6 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executor; import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiFunction; import java.util.function.Consumer; import java.util.function.Function; @@ -70,22 +68,14 @@ */ abstract class AbstractSearchAsyncAction extends AsyncSearchContext { static final float DEFAULT_INDEX_BOOST = 1.0f; - private final Logger logger; - private final SearchTransportService searchTransportService; - private final Executor executor; + protected final Logger logger; - /** - * Used by subclasses to resolve node ids to DiscoveryNodes. - **/ - private final BiFunction nodeIdToConnection; - private final SearchTask task; private final long clusterStateVersion; private final TransportVersion minTransportVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; private final SetOnce> shardFailures = new SetOnce<>(); private final Object shardFailuresMutex = new Object(); - private final AtomicInteger successfulOps = new AtomicInteger(); private final SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; @@ -108,7 +98,6 @@ abstract class AbstractSearchAsyncAction exten private final int maxConcurrentRequestsPerNode; private final Map pendingExecutionsPerNode = new ConcurrentHashMap<>(); private final boolean throttleConcurrentRequests; - private final AtomicBoolean requestCancelled = new AtomicBoolean(); // protected for tests protected final String name; @@ -132,7 +121,7 @@ abstract class AbstractSearchAsyncAction exten int maxConcurrentRequestsPerNode, SearchResponse.Clusters clusters ) { - super(request, resultConsumer, namedWriteableRegistry, listener); + super(request, resultConsumer, namedWriteableRegistry, listener, task, searchTransportService, executor, nodeIdToConnection); this.name = name; final List toSkipIterators = new ArrayList<>(); final List iterators = new ArrayList<>(); @@ -156,10 +145,6 @@ abstract class AbstractSearchAsyncAction exten this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size(); this.timeProvider = timeProvider; this.logger = logger; - this.searchTransportService = searchTransportService; - this.executor = executor; - this.task = task; - this.nodeIdToConnection = nodeIdToConnection; this.concreteIndexBoosts = concreteIndexBoosts; this.clusterStateVersion = clusterState.version(); this.minTransportVersion = clusterState.getMinTransportVersion(); @@ -307,7 +292,7 @@ private void doPerformPhaseOnShard(int shardIndex, SearchShardIterator shardIt, public void innerOnResponse(Result result) { try { releasable.close(); - onShardResult(result, shardIt); + onShardResult(result); } catch (Exception exc) { onShardFailure(shardIndex, shard, shardIt, exc); } @@ -409,14 +394,7 @@ public void executeNextPhase(String currentPhase, Supplier nextPhas clusterStateVersion ); } - try { - nextPhase.run(); - } catch (RuntimeException e) { - if (logger.isDebugEnabled()) { - logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, nextPhase.getName()), e); - } - onPhaseFailure(nextPhase.getName(), "", e); - } + executePhase(nextPhase); } } @@ -514,16 +492,15 @@ static boolean isTaskCancelledException(Exception e) { /** * Executed once for every successful shard level request. * @param result the result returned form the shard - * @param shardIt the shard iterator */ - protected void onShardResult(Result result, SearchShardIterator shardIt) { + protected void onShardResult(Result result) { assert result.getShardIndex() != -1 : "shard index is not set"; assert result.getSearchShardTarget() != null : "search shard target must not be null"; if (hasShardResponse == false) { hasShardResponse = true; } if (logger.isTraceEnabled()) { - logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); + logger.trace("got first-phase result from {}", result.getSearchShardTarget()); } results.consumeResult(result, () -> onShardResultConsumed(result)); } @@ -551,13 +528,6 @@ private void successfulShardExecution() { } } - /** - * Returns the currently executing search task - */ - public final SearchTask getTask() { - return task; - } - /** * Returns the targeted {@link OriginalIndices} for the provided {@code shardIndex}. */ @@ -652,7 +622,7 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C /** * Executed once all shard results have been received and processed * @see #onShardFailure(int, SearchShardTarget, Exception) - * @see #onShardResult(SearchPhaseResult, SearchShardIterator) + * @see #onShardResult(SearchPhaseResult) */ private void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() executeNextPhase(name, this::getNextPhase); @@ -667,13 +637,6 @@ public final Transport.Connection getConnection(String clusterAlias, String node return nodeIdToConnection.apply(clusterAlias, nodeId); } - /** - * Returns the {@link SearchTransportService} to send shard request to other nodes - */ - public SearchTransportService getSearchTransport() { - return searchTransportService; - } - public final void execute(Runnable command) { executor.execute(command); } diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index b39b01ef1a09b..44821b532b64d 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -26,8 +26,14 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiFunction; import java.util.function.Supplier; +import static org.elasticsearch.core.Strings.format; + public abstract class AsyncSearchContext { private static final Logger logger = LogManager.getLogger(AsyncSearchContext.class); @@ -45,16 +51,38 @@ public abstract class AsyncSearchContext { // protected for tests protected final List releasables = new ArrayList<>(); + protected final AtomicBoolean requestCancelled = new AtomicBoolean(); + + protected final SearchTask task; + + protected final AtomicInteger successfulOps = new AtomicInteger(); + + protected final SearchTransportService searchTransportService; + protected final Executor executor; + + /** + * Used by subclasses to resolve node ids to DiscoveryNodes. + **/ + protected final BiFunction nodeIdToConnection; + protected AsyncSearchContext( SearchRequest request, SearchPhaseResults results, NamedWriteableRegistry namedWriteableRegistry, - ActionListener listener + ActionListener listener, + SearchTask task, + SearchTransportService searchTransportService, + Executor executor, + BiFunction nodeIdToConnection ) { this.request = request; this.results = results; this.namedWriteableRegistry = namedWriteableRegistry; this.listener = ActionListener.runAfter(listener, () -> Releasables.close(releasables)); + this.task = task; + this.searchTransportService = searchTransportService; + this.executor = executor; + this.nodeIdToConnection = nodeIdToConnection; ; // register the release of the query consumer to free up the circuit breaker memory // at the end of the search @@ -92,9 +120,19 @@ public final SearchRequest getRequest() { abstract void sendSearchResponse(SearchResponseSections internalSearchResponse, AtomicArray queryResults); - abstract SearchTransportService getSearchTransport(); + /** + * Returns the {@link SearchTransportService} to send shard request to other nodes + */ + public SearchTransportService getSearchTransport() { + return searchTransportService; + } - abstract SearchTask getTask(); + /** + * Returns the currently executing search task + */ + public final SearchTask getTask() { + return task; + } abstract void onPhaseFailure(String phase, String msg, Throwable cause); @@ -147,4 +185,16 @@ protected final void raisePhaseFailure(SearchPhaseExecutionException exception) public boolean isPartOfPointInTime(ShardSearchContextId contextId) { return isPartOfPIT(namedWriteableRegistry, request, contextId); } + + protected final void executePhase(SearchPhase phase) { + try { + phase.run(); + } catch (Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, phase.getName()), e); + } + onPhaseFailure(phase.getName(), "", e); + } + } + } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index d7df44c021f67..b438a5133fff4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -98,27 +98,17 @@ public class SearchQueryThenFetchAsyncAction e private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class); - private final SearchTransportService searchTransportService; - private final Executor executor; - - /** - * Used by subclasses to resolve node ids to DiscoveryNodes. - **/ - private final BiFunction nodeIdToConnection; - private final SearchTask task; private final TransportVersion minNodeVersion; private final Map aliasFilter; private final Map concreteIndexBoosts; private final SetOnce> shardFailures = new SetOnce<>(); private final Object shardFailuresMutex = new Object(); - private final AtomicInteger successfulOps; private final TransportSearchAction.SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; protected final GroupShardsIterator toSkipShardsIts; protected final GroupShardsIterator shardsIts; private final SearchShardIterator[] shardIterators; - private final AtomicBoolean requestCancelled = new AtomicBoolean(); private static final VarHandle OUTSTANDING_SHARDS; @@ -158,7 +148,7 @@ public class SearchQueryThenFetchAsyncAction e SearchResponse.Clusters clusters, Client client ) { - super(request, resultConsumer, namedWriteableRegistry, listener); + super(request, resultConsumer, namedWriteableRegistry, listener, task, searchTransportService, executor, nodeIdToConnection); final List toSkipIterators = new ArrayList<>(); final List iterators = new ArrayList<>(); for (final SearchShardIterator iterator : shardsIts) { @@ -169,7 +159,7 @@ public class SearchQueryThenFetchAsyncAction e } } this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); - this.successfulOps = new AtomicInteger(toSkipIterators.size()); + this.successfulOps.setRelease(toSkipIterators.size()); this.shardsIts = new GroupShardsIterator<>(iterators); this.shardIterators = iterators.toArray(new SearchShardIterator[0]); @@ -179,10 +169,6 @@ public class SearchQueryThenFetchAsyncAction e // consistent between two requests that target the same shards. Arrays.sort(shardIterators); this.timeProvider = timeProvider; - this.searchTransportService = searchTransportService; - this.executor = executor; - this.task = task; - this.nodeIdToConnection = nodeIdToConnection; this.concreteIndexBoosts = concreteIndexBoosts; this.minNodeVersion = clusterState.getMinTransportVersion(); this.aliasFilter = aliasFilter; @@ -255,16 +241,6 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At } } - @Override - public SearchTransportService getSearchTransport() { - return searchTransportService; - } - - @Override - public SearchTask getTask() { - return task; - } - private SearchResponse buildSearchResponse( SearchResponseSections internalSearchResponse, ShardSearchFailure[] failures, @@ -920,17 +896,6 @@ public void executeNextPhase(String currentPhase, Supplier nextPhas } } - private void executePhase(SearchPhase phase) { - try { - phase.run(); - } catch (Exception e) { - if (logger.isDebugEnabled()) { - logger.debug(() -> format("Failed to execute [%s] while moving to [%s] phase", request, phase.getName()), e); - } - onPhaseFailure(phase.getName(), "", e); - } - } - /** * This method will communicate a fatal phase failure back to the user. In contrast to a shard failure * will this method immediately fail the search request and return the failure to the issuer of the request diff --git a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java index 5d6cf92bfcc7d..e79f39bc79664 100644 --- a/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java +++ b/server/src/test/java/org/elasticsearch/action/search/MockSearchPhaseContext.java @@ -151,7 +151,7 @@ protected void executePhaseOnShard( SearchActionListener listener ) { onShardResult(new SearchPhaseResult() { - }, shardIt); + }); } @Override From f4e86bfc361cc4f5e0c73ba48db7de015c4d4c35 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 15:38:25 +0100 Subject: [PATCH 106/132] extract --- .../search/AbstractSearchAsyncAction.java | 75 ++++--------------- .../action/search/AsyncSearchContext.java | 59 ++++++++++++++- .../SearchQueryThenFetchAsyncAction.java | 63 +++------------- .../AbstractSearchAsyncActionTests.java | 7 +- 4 files changed, 86 insertions(+), 118 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 9d8ba0d16c5b8..b341473ac842a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -42,9 +42,6 @@ import java.lang.invoke.MethodHandles; import java.lang.invoke.VarHandle; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; @@ -79,10 +76,6 @@ abstract class AbstractSearchAsyncAction exten private final SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; - protected final GroupShardsIterator toSkipShardsIts; - protected final GroupShardsIterator shardsIts; - private final SearchShardIterator[] shardIterators; - private static final VarHandle OUTSTANDING_SHARDS; static { @@ -121,25 +114,19 @@ abstract class AbstractSearchAsyncAction exten int maxConcurrentRequestsPerNode, SearchResponse.Clusters clusters ) { - super(request, resultConsumer, namedWriteableRegistry, listener, task, searchTransportService, executor, nodeIdToConnection); + super( + request, + resultConsumer, + namedWriteableRegistry, + listener, + task, + searchTransportService, + executor, + nodeIdToConnection, + shardsIts + ); this.name = name; - final List toSkipIterators = new ArrayList<>(); - final List iterators = new ArrayList<>(); - for (final SearchShardIterator iterator : shardsIts) { - if (iterator.skip()) { - toSkipIterators.add(iterator); - } else { - iterators.add(iterator); - } - } - this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); - this.shardsIts = new GroupShardsIterator<>(iterators); - OUTSTANDING_SHARDS.setRelease(this, shardsIts.size()); - this.shardIterators = iterators.toArray(new SearchShardIterator[0]); - // we later compute the shard index based on the natural order of the shards - // that participate in the search request. This means that this number is - // consistent between two requests that target the same shards. - Arrays.sort(shardIterators); + OUTSTANDING_SHARDS.setRelease(this, shardIterators.length); this.maxConcurrentRequestsPerNode = maxConcurrentRequestsPerNode; // in the case were we have less shards than maxConcurrentRequestsPerNode we don't need to throttle this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size(); @@ -243,9 +230,9 @@ public final void start() { } private void run() { - for (final SearchShardIterator iterator : toSkipShardsIts) { - assert iterator.skip(); - skipShard(iterator); + if ((int) OUTSTANDING_SHARDS.getAcquire(this) == 0) { + onPhaseDone(); + return; } final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); for (int i = 0; i < shardIterators.length; i++) { @@ -268,12 +255,6 @@ private void run() { } } - void skipShard(SearchShardIterator iterator) { - successfulOps.incrementAndGet(); - assert iterator.skip(); - successfulShardExecution(); - } - private void performPhaseOnShard(final int shardIndex, final SearchShardIterator shardIt, final SearchShardTarget shard) { if (throttleConcurrentRequests) { var pendingExecutions = pendingExecutionsPerNode.computeIfAbsent( @@ -354,31 +335,7 @@ public void executeNextPhase(String currentPhase, Supplier nextPhas Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; if (allowPartialResults == false && successfulOps.get() != numShards) { - // check if there are actual failures in the atomic array since - // successful retries can reset the failures to null - if (shardSearchFailures.length > 0) { - if (logger.isDebugEnabled()) { - int numShardFailures = shardSearchFailures.length; - shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); - Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase), cause); - } - onPhaseFailure(currentPhase, "Partial shards failure", null); - } else { - int discrepancy = numShards - successfulOps.get(); - assert discrepancy > 0 : "discrepancy: " + discrepancy; - if (logger.isDebugEnabled()) { - logger.debug( - "Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})", - discrepancy, - successfulOps.get(), - toSkipShardsIts.size(), - numShards, - currentPhase - ); - } - onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); - } + handleNotAllSucceeded(currentPhase, shardSearchFailures, numShards); return; } var nextPhase = nextPhaseSupplier.get(); diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 44821b532b64d..e6339503fc8f5 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -12,8 +12,12 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.util.SetOnce; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; +import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; @@ -25,6 +29,7 @@ import org.elasticsearch.transport.Transport; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; @@ -60,6 +65,10 @@ public abstract class AsyncSearchContext { protected final SearchTransportService searchTransportService; protected final Executor executor; + protected final GroupShardsIterator toSkipShardsIts; + protected final GroupShardsIterator shardsIts; + protected final SearchShardIterator[] shardIterators; + /** * Used by subclasses to resolve node ids to DiscoveryNodes. **/ @@ -73,8 +82,27 @@ protected AsyncSearchContext( SearchTask task, SearchTransportService searchTransportService, Executor executor, - BiFunction nodeIdToConnection + BiFunction nodeIdToConnection, + GroupShardsIterator shardsIts ) { + final List toSkipIterators = new ArrayList<>(); + final List iterators = new ArrayList<>(); + for (final SearchShardIterator iterator : shardsIts) { + if (iterator.skip()) { + toSkipIterators.add(iterator); + } else { + iterators.add(iterator); + } + } + this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); + this.successfulOps.setRelease(toSkipIterators.size()); + this.shardsIts = new GroupShardsIterator<>(iterators); + + this.shardIterators = iterators.toArray(new SearchShardIterator[0]); + // we later compute the shard index based on the natural order of the shards + // that participate in the search request. This means that this number is + // consistent between two requests that target the same shards. + Arrays.sort(shardIterators); this.request = request; this.results = results; this.namedWriteableRegistry = namedWriteableRegistry; @@ -83,7 +111,6 @@ protected AsyncSearchContext( this.searchTransportService = searchTransportService; this.executor = executor; this.nodeIdToConnection = nodeIdToConnection; - ; // register the release of the query consumer to free up the circuit breaker memory // at the end of the search addReleasable(results); @@ -197,4 +224,32 @@ protected final void executePhase(SearchPhase phase) { } } + protected final void handleNotAllSucceeded(String currentPhase, ShardOperationFailedException[] shardSearchFailures, int numShards) { + // check if there are actual failures in the atomic array since + // successful retries can reset the failures to null + if (shardSearchFailures.length > 0) { + if (logger.isDebugEnabled()) { + int numShardFailures = shardSearchFailures.length; + shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); + Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; + logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase), cause); + } + onPhaseFailure(currentPhase, "Partial shards failure", null); + } else { + int discrepancy = numShards - successfulOps.get(); + assert discrepancy > 0 : "discrepancy: " + discrepancy; + if (logger.isDebugEnabled()) { + logger.debug( + "Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})", + discrepancy, + successfulOps.get(), + toSkipShardsIts.size(), + numShards, + currentPhase + ); + } + onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); + } + } + } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index b438a5133fff4..2bcdf2789e14c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -106,10 +106,6 @@ public class SearchQueryThenFetchAsyncAction e private final TransportSearchAction.SearchTimeProvider timeProvider; private final SearchResponse.Clusters clusters; - protected final GroupShardsIterator toSkipShardsIts; - protected final GroupShardsIterator shardsIts; - private final SearchShardIterator[] shardIterators; - private static final VarHandle OUTSTANDING_SHARDS; static { @@ -148,33 +144,22 @@ public class SearchQueryThenFetchAsyncAction e SearchResponse.Clusters clusters, Client client ) { - super(request, resultConsumer, namedWriteableRegistry, listener, task, searchTransportService, executor, nodeIdToConnection); - final List toSkipIterators = new ArrayList<>(); - final List iterators = new ArrayList<>(); - for (final SearchShardIterator iterator : shardsIts) { - if (iterator.skip()) { - toSkipIterators.add(iterator); - } else { - iterators.add(iterator); - } - } - this.toSkipShardsIts = new GroupShardsIterator<>(toSkipIterators); - this.successfulOps.setRelease(toSkipIterators.size()); - this.shardsIts = new GroupShardsIterator<>(iterators); - - this.shardIterators = iterators.toArray(new SearchShardIterator[0]); + super( + request, + resultConsumer, + namedWriteableRegistry, + listener, + task, + searchTransportService, + executor, + nodeIdToConnection, + shardsIts + ); outstandingShards = shardIterators.length; - // we compute the shard index based on the natural order of the shards - // that participate in the search request. This means that this number is - // consistent between two requests that target the same shards. - Arrays.sort(shardIterators); this.timeProvider = timeProvider; this.concreteIndexBoosts = concreteIndexBoosts; this.minNodeVersion = clusterState.getMinTransportVersion(); this.aliasFilter = aliasFilter; - // register the release of the query consumer to free up the circuit breaker memory - // at the end of the search - releasables.add(resultConsumer); this.clusters = clusters; this.topDocsSize = getTopDocsSize(request); this.trackTotalHitsUpTo = request.resolveTrackTotalHitsUpTo(); @@ -856,31 +841,7 @@ public void executeNextPhase(String currentPhase, Supplier nextPhas Boolean allowPartialResults = request.allowPartialSearchResults(); assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults"; if (allowPartialResults == false && successfulOps.get() != numShards) { - // check if there are actual failures in the atomic array since - // successful retries can reset the failures to null - if (shardSearchFailures.length > 0) { - if (logger.isDebugEnabled()) { - int numShardFailures = shardSearchFailures.length; - shardSearchFailures = ExceptionsHelper.groupBy(shardSearchFailures); - Throwable cause = ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0]; - logger.debug(() -> format("%s shards failed for phase: [%s]", numShardFailures, currentPhase), cause); - } - onPhaseFailure(currentPhase, "Partial shards failure", null); - } else { - int discrepancy = numShards - successfulOps.get(); - assert discrepancy > 0 : "discrepancy: " + discrepancy; - if (logger.isDebugEnabled()) { - logger.debug( - "Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})", - discrepancy, - successfulOps.get(), - toSkipShardsIts.size(), - numShards, - currentPhase - ); - } - onPhaseFailure(currentPhase, "Partial shards failure (" + discrepancy + " shards unavailable)", null); - } + handleNotAllSucceeded(currentPhase, shardSearchFailures, numShards); return; } var nextPhase = nextPhaseSupplier.get(); diff --git a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java index 47ff4ca6f0600..1ac8ab0d516e8 100644 --- a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java @@ -229,13 +229,8 @@ public void testShardNotAvailableWithDisallowPartialFailures() { SearchShardIterator skipIterator = new SearchShardIterator(null, null, Collections.emptyList(), null); skipIterator.skip(true); skipIterator.reset(); - action.skipShard(skipIterator); + action.start(); assertThat(exception.get(), instanceOf(SearchPhaseExecutionException.class)); - SearchPhaseExecutionException searchPhaseExecutionException = (SearchPhaseExecutionException) exception.get(); - assertEquals("Partial shards failure (" + (numShards - 1) + " shards unavailable)", searchPhaseExecutionException.getMessage()); - assertEquals("test", searchPhaseExecutionException.getPhaseName()); - assertEquals(0, searchPhaseExecutionException.shardFailures().length); - assertEquals(0, searchPhaseExecutionException.getSuppressed().length); } private static ArraySearchPhaseResults phaseResults( From a02d2f1fac94ce526825f4c8bb0aa967fd0a5000 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 15:56:38 +0100 Subject: [PATCH 107/132] extract --- .../action/search/AbstractSearchAsyncAction.java | 9 --------- .../elasticsearch/action/search/AsyncSearchContext.java | 4 +++- .../action/search/SearchQueryThenFetchAsyncAction.java | 5 ----- 3 files changed, 3 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index b341473ac842a..6eed22b2bcfbe 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -585,15 +585,6 @@ private void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() executeNextPhase(name, this::getNextPhase); } - /** - * Returns a connection to the node if connected otherwise and {@link org.elasticsearch.transport.ConnectTransportException} will be - * thrown. - */ - @Override - public final Transport.Connection getConnection(String clusterAlias, String nodeId) { - return nodeIdToConnection.apply(clusterAlias, nodeId); - } - public final void execute(Runnable command) { executor.execute(command); } diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index e6339503fc8f5..c51fcb6ea0403 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -174,7 +174,9 @@ public final void addReleasable(Releasable releasable) { abstract void onShardFailure(int shardIndex, SearchShardTarget shard, Exception e); - abstract Transport.Connection getConnection(String clusterAlias, String nodeId); + public final Transport.Connection getConnection(String clusterAlias, String nodeId) { + return nodeIdToConnection.apply(clusterAlias, nodeId); + } abstract OriginalIndices getOriginalIndices(int shardIndex); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 2bcdf2789e14c..a52828d87d3fe 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -733,11 +733,6 @@ public void onFailure(Exception e) { ); } - @Override - public final Transport.Connection getConnection(String clusterAlias, String nodeId) { - return nodeIdToConnection.apply(clusterAlias, nodeId); - } - private void onShardFailure(final int shardIndex, SearchShardTarget shard, final SearchShardIterator shardIt, Exception e) { // we always add the shard failure for a specific shard instance // we do make sure to clean it on a successful response from a shard From d53594a0a18041ce202baa5877de809379c7fe9b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 16:33:21 +0100 Subject: [PATCH 108/132] cleanup --- .../search/AbstractSearchAsyncAction.java | 37 +++------------- .../action/search/AsyncSearchContext.java | 40 ++++++++++++++++- .../SearchQueryThenFetchAsyncAction.java | 43 ++++++------------- .../action/search/SearchAsyncActionTests.java | 12 +++--- 4 files changed, 63 insertions(+), 69 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 6eed22b2bcfbe..3b719e8121d84 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -10,13 +10,10 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; -import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.search.TransportSearchAction.SearchTimeProvider; import org.elasticsearch.action.support.SubscribableListener; @@ -68,13 +65,6 @@ abstract class AbstractSearchAsyncAction exten protected final Logger logger; private final long clusterStateVersion; - private final TransportVersion minTransportVersion; - private final Map aliasFilter; - private final Map concreteIndexBoosts; - private final SetOnce> shardFailures = new SetOnce<>(); - private final Object shardFailuresMutex = new Object(); - private final SearchTimeProvider timeProvider; - private final SearchResponse.Clusters clusters; private static final VarHandle OUTSTANDING_SHARDS; @@ -123,20 +113,20 @@ abstract class AbstractSearchAsyncAction exten searchTransportService, executor, nodeIdToConnection, - shardsIts + shardsIts, + aliasFilter, + concreteIndexBoosts, + timeProvider, + clusterState, + clusters ); this.name = name; OUTSTANDING_SHARDS.setRelease(this, shardIterators.length); this.maxConcurrentRequestsPerNode = maxConcurrentRequestsPerNode; // in the case were we have less shards than maxConcurrentRequestsPerNode we don't need to throttle this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size(); - this.timeProvider = timeProvider; this.logger = logger; - this.concreteIndexBoosts = concreteIndexBoosts; this.clusterStateVersion = clusterState.version(); - this.minTransportVersion = clusterState.getMinTransportVersion(); - this.aliasFilter = aliasFilter; - this.clusters = clusters; } protected void notifyListShards( @@ -485,13 +475,6 @@ private void successfulShardExecution() { } } - /** - * Returns the targeted {@link OriginalIndices} for the provided {@code shardIndex}. - */ - public OriginalIndices getOriginalIndices(int shardIndex) { - return shardIterators[shardIndex].getOriginalIndices(); - } - private SearchResponse buildSearchResponse( SearchResponseSections internalSearchResponse, ShardSearchFailure[] failures, @@ -538,13 +521,7 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At if (buildPointInTimeFromSearchResults()) { searchContextId = SearchContextId.encode(queryResults.asList(), aliasFilter, minTransportVersion, failures); } else { - if (request.source() != null - && request.source().pointInTimeBuilder() != null - && request.source().pointInTimeBuilder().singleSession() == false) { - searchContextId = request.source().pointInTimeBuilder().getEncodedId(); - } else { - searchContextId = null; - } + searchContextId = buildSearchContextId(); } ActionListener.respondAndRelease(listener, buildSearchResponse(internalSearchResponse, failures, scrollId, searchContextId)); } diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index c51fcb6ea0403..9bc8d92e66908 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -14,10 +14,13 @@ import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.TransportVersion; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; @@ -25,12 +28,14 @@ import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.builder.PointInTimeBuilder; +import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.transport.Transport; import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -69,6 +74,15 @@ public abstract class AsyncSearchContext { protected final GroupShardsIterator shardsIts; protected final SearchShardIterator[] shardIterators; + protected final SetOnce> shardFailures = new SetOnce<>(); + protected final Object shardFailuresMutex = new Object(); + + protected final TransportVersion minTransportVersion; + protected final Map aliasFilter; + protected final Map concreteIndexBoosts; + protected final TransportSearchAction.SearchTimeProvider timeProvider; + protected final SearchResponse.Clusters clusters; + /** * Used by subclasses to resolve node ids to DiscoveryNodes. **/ @@ -83,7 +97,12 @@ protected AsyncSearchContext( SearchTransportService searchTransportService, Executor executor, BiFunction nodeIdToConnection, - GroupShardsIterator shardsIts + GroupShardsIterator shardsIts, + Map aliasFilter, + Map concreteIndexBoosts, + TransportSearchAction.SearchTimeProvider timeProvider, + ClusterState clusterState, + SearchResponse.Clusters clusters ) { final List toSkipIterators = new ArrayList<>(); final List iterators = new ArrayList<>(); @@ -114,6 +133,12 @@ protected AsyncSearchContext( // register the release of the query consumer to free up the circuit breaker memory // at the end of the search addReleasable(results); + + this.timeProvider = timeProvider; + this.concreteIndexBoosts = concreteIndexBoosts; + this.minTransportVersion = clusterState.getMinTransportVersion(); + this.aliasFilter = aliasFilter; + this.clusters = clusters; } static ShardSearchFailure[] buildShardFailures(SetOnce> shardFailuresRef) { @@ -178,7 +203,12 @@ public final Transport.Connection getConnection(String clusterAlias, String node return nodeIdToConnection.apply(clusterAlias, nodeId); } - abstract OriginalIndices getOriginalIndices(int shardIndex); + /** + * Returns the targeted {@link OriginalIndices} for the provided {@code shardIndex}. + */ + public OriginalIndices getOriginalIndices(int shardIndex) { + return shardIterators[shardIndex].getOriginalIndices(); + } abstract void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection); @@ -254,4 +284,10 @@ protected final void handleNotAllSucceeded(String currentPhase, ShardOperationFa } } + protected BytesReference buildSearchContextId() { + var source = request.source(); + return source != null && source.pointInTimeBuilder() != null && source.pointInTimeBuilder().singleSession() == false + ? source.pointInTimeBuilder().getEncodedId() + : null; + } } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index a52828d87d3fe..494446f672091 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -13,10 +13,8 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.search.ScoreDoc; import org.apache.lucene.search.TopFieldDocs; -import org.apache.lucene.util.SetOnce; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; -import org.elasticsearch.TransportVersion; import org.elasticsearch.TransportVersions; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.IndicesRequest; @@ -98,14 +96,6 @@ public class SearchQueryThenFetchAsyncAction e private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class); - private final TransportVersion minNodeVersion; - private final Map aliasFilter; - private final Map concreteIndexBoosts; - private final SetOnce> shardFailures = new SetOnce<>(); - private final Object shardFailuresMutex = new Object(); - private final TransportSearchAction.SearchTimeProvider timeProvider; - private final SearchResponse.Clusters clusters; - private static final VarHandle OUTSTANDING_SHARDS; static { @@ -153,19 +143,19 @@ public class SearchQueryThenFetchAsyncAction e searchTransportService, executor, nodeIdToConnection, - shardsIts + shardsIts, + aliasFilter, + concreteIndexBoosts, + timeProvider, + clusterState, + clusters ); outstandingShards = shardIterators.length; - this.timeProvider = timeProvider; - this.concreteIndexBoosts = concreteIndexBoosts; - this.minNodeVersion = clusterState.getMinTransportVersion(); - this.aliasFilter = aliasFilter; - this.clusters = clusters; this.topDocsSize = getTopDocsSize(request); this.trackTotalHitsUpTo = request.resolveTrackTotalHitsUpTo(); - this.progressListener = task.getProgressListener(); this.client = client; + this.progressListener = task.getProgressListener(); // don't build the SearchShard list (can be expensive) if the SearchProgressListener won't use it if (progressListener != SearchProgressListener.NOOP) { notifyListShards(progressListener, clusters, request.source()); @@ -215,14 +205,10 @@ public void sendSearchResponse(SearchResponseSections internalSearchResponse, At raisePhaseFailure(new SearchPhaseExecutionException("", "Shard failures", null, failures)); } else { final String scrollId = request.scroll() != null ? TransportSearchHelper.buildScrollId(queryResults) : null; - var source = request.source(); - final BytesReference searchContextId; - if (source != null && source.pointInTimeBuilder() != null && source.pointInTimeBuilder().singleSession() == false) { - searchContextId = source.pointInTimeBuilder().getEncodedId(); - } else { - searchContextId = null; - } - ActionListener.respondAndRelease(listener, buildSearchResponse(internalSearchResponse, failures, scrollId, searchContextId)); + ActionListener.respondAndRelease( + listener, + buildSearchResponse(internalSearchResponse, failures, scrollId, buildSearchContextId()) + ); } } @@ -508,11 +494,6 @@ protected SearchPhase getNextPhase() { return nextPhase(client, this, results, null); } - @Override - public OriginalIndices getOriginalIndices(int shardIndex) { - return shardIterators[shardIndex].getOriginalIndices(); - } - private static ShardSearchRequest rewriteShardSearchRequest( BottomSortValuesCollector bottomSortCollector, int trackTotalHitsUpTo, @@ -545,7 +526,7 @@ private void run() { finish(); return; } - final boolean supportsBatchedQuery = minNodeVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); + final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); AbstractSearchAsyncAction.doCheckNoMissingShards(NAME, request, shardsIts, AbstractSearchAsyncAction::makeMissingShardsError); final String localNodeId = searchTransportService.transportService().getLocalNode().getId(); diff --git a/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java index 7e9e6f623cab0..4b3a031f45e46 100644 --- a/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/SearchAsyncActionTests.java @@ -166,13 +166,13 @@ public void testLimitConcurrentShardRequests() throws InterruptedException { request.setMaxConcurrentShardRequests(numConcurrent); boolean doReplicas = randomBoolean(); int numShards = randomIntBetween(5, 10); - Boolean[] shardFailures = new Boolean[numShards]; + Boolean[] sFailures = new Boolean[numShards]; // at least one response otherwise the entire request fails - shardFailures[randomIntBetween(0, shardFailures.length - 1)] = false; - for (int i = 0; i < shardFailures.length; i++) { - if (shardFailures[i] == null) { + sFailures[randomIntBetween(0, sFailures.length - 1)] = false; + for (int i = 0; i < sFailures.length; i++) { + if (sFailures[i] == null) { boolean failure = randomBoolean(); - shardFailures[i] = failure; + sFailures[i] = failure; } } CountDownLatch latch = new CountDownLatch(1); @@ -240,7 +240,7 @@ protected void executePhaseOnShard( connection.getNode() ); try { - if (shardFailures[shardIt.shardId().id()]) { + if (sFailures[shardIt.shardId().id()]) { listener.onFailure(new RuntimeException()); } else { listener.onResponse(testSearchPhaseResult); From f1d35db00bb8df67f3ce0dd16577e8e283b07f89 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 23 Jan 2025 16:54:28 +0100 Subject: [PATCH 109/132] cleanup --- .../action/search/AbstractSearchAsyncAction.java | 15 --------------- .../action/search/AsyncSearchContext.java | 15 +++++++++++++++ .../search/SearchQueryThenFetchAsyncAction.java | 15 --------------- 3 files changed, 15 insertions(+), 30 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index 3b719e8121d84..bc559314ea520 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -29,7 +29,6 @@ import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchContextId; @@ -129,20 +128,6 @@ abstract class AbstractSearchAsyncAction exten this.clusterStateVersion = clusterState.version(); } - protected void notifyListShards( - SearchProgressListener progressListener, - SearchResponse.Clusters clusters, - SearchSourceBuilder sourceBuilder - ) { - progressListener.notifyListShards( - SearchProgressListener.buildSearchShards(this.shardsIts), - SearchProgressListener.buildSearchShards(toSkipShardsIts), - clusters, - sourceBuilder == null || sourceBuilder.size() > 0, - timeProvider - ); - } - protected String missingShardsErrorMessage(StringBuilder missingShards) { return makeMissingShardsError(missingShards); } diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 9bc8d92e66908..fdce2541afcdd 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -28,6 +28,7 @@ import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.builder.PointInTimeBuilder; +import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.transport.Transport; @@ -141,6 +142,20 @@ protected AsyncSearchContext( this.clusters = clusters; } + protected void notifyListShards( + SearchProgressListener progressListener, + SearchResponse.Clusters clusters, + SearchSourceBuilder sourceBuilder + ) { + progressListener.notifyListShards( + SearchProgressListener.buildSearchShards(this.shardsIts), + SearchProgressListener.buildSearchShards(toSkipShardsIts), + clusters, + sourceBuilder == null || sourceBuilder.size() > 0, + timeProvider + ); + } + static ShardSearchFailure[] buildShardFailures(SetOnce> shardFailuresRef) { AtomicArray shardFailures = shardFailuresRef.get(); if (shardFailures == null) { diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 494446f672091..108f079e182e3 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -48,7 +48,6 @@ import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; -import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.dfs.AggregatedDfs; import org.elasticsearch.search.internal.AliasFilter; import org.elasticsearch.search.internal.SearchContext; @@ -648,20 +647,6 @@ private void failOnUnavailable(int shardIndex, SearchShardIterator shardIt) { onShardFailure(shardIndex, unassignedShard, shardIt, new NoShardAvailableActionException(shardIt.shardId())); } - protected void notifyListShards( - SearchProgressListener progressListener, - SearchResponse.Clusters clusters, - SearchSourceBuilder sourceBuilder - ) { - progressListener.notifyListShards( - SearchProgressListener.buildSearchShards(this.shardsIts), - SearchProgressListener.buildSearchShards(toSkipShardsIts), - clusters, - sourceBuilder == null || sourceBuilder.size() > 0, - timeProvider - ); - } - protected void performPhaseOnShard(final int shardIndex, final SearchShardIterator shardIt, final SearchShardTarget shard) { final Transport.Connection connection; try { From 901666b7abac2a8c748a7e1d7f18c73cb7db4ece Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 26 Jan 2025 12:55:06 +0100 Subject: [PATCH 110/132] bck --- .../search/AbstractSearchAsyncAction.java | 95 ++----------------- .../action/search/AsyncSearchContext.java | 87 +++++++++++++++++ .../SearchQueryThenFetchAsyncAction.java | 85 ++--------------- 3 files changed, 103 insertions(+), 164 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index bc559314ea520..f38aef6fe3a01 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -26,18 +26,13 @@ import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.internal.AliasFilter; -import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchContextId; import org.elasticsearch.search.internal.ShardSearchRequest; -import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.transport.Transport; -import java.lang.invoke.MethodHandles; -import java.lang.invoke.VarHandle; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; @@ -64,19 +59,6 @@ abstract class AbstractSearchAsyncAction exten protected final Logger logger; private final long clusterStateVersion; - - private static final VarHandle OUTSTANDING_SHARDS; - - static { - try { - OUTSTANDING_SHARDS = MethodHandles.lookup().findVarHandle(AbstractSearchAsyncAction.class, "outstandingShards", int.class); - } catch (Exception e) { - throw new ExceptionInInitializerError(e); - } - } - - @SuppressWarnings("unused") // only accessed via #OUTSTANDING_SHARDS - private int outstandingShards; private final int maxConcurrentRequestsPerNode; private final Map pendingExecutionsPerNode = new ConcurrentHashMap<>(); private final boolean throttleConcurrentRequests; @@ -120,7 +102,6 @@ abstract class AbstractSearchAsyncAction exten clusters ); this.name = name; - OUTSTANDING_SHARDS.setRelease(this, shardIterators.length); this.maxConcurrentRequestsPerNode = maxConcurrentRequestsPerNode; // in the case were we have less shards than maxConcurrentRequestsPerNode we don't need to throttle this.throttleConcurrentRequests = maxConcurrentRequestsPerNode < shardsIts.size(); @@ -181,17 +162,7 @@ long buildTookInMillis() { */ public final void start() { if (results.getNumShards() == 0) { - // no search shards to search on, bail with empty response - // (it happens with search across _all with no indices around and consistent with broadcast operations) - int trackTotalHitsUpTo = request.source() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO - : request.source().trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO - : request.source().trackTotalHitsUpTo(); - // total hits is null in the response if the tracking of total hits is disabled - boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; - sendSearchResponse( - withTotalHits ? SearchResponseSections.EMPTY_WITH_TOTAL_HITS : SearchResponseSections.EMPTY_WITHOUT_TOTAL_HITS, - new AtomicArray<>(0) - ); + sendZeroShardsResponse(); return; } try { @@ -205,7 +176,7 @@ public final void start() { } private void run() { - if ((int) OUTSTANDING_SHARDS.getAcquire(this) == 0) { + if (shardsIts.size() == 0) { onPhaseDone(); return; } @@ -338,26 +309,13 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final final boolean lastShard = nextShard == null; logger.debug(() -> format("%s: Failed to execute [%s] lastShard [%s]", shard, request, lastShard), e); if (lastShard) { - if (request.allowPartialSearchResults() == false) { - if (requestCancelled.compareAndSet(false, true)) { - try { - searchTransportService.cancelSearchTask( - task.getId(), - "partial results are not allowed and at least one shard has failed" - ); - } catch (Exception cancelFailure) { - logger.debug("Failed to cancel search request", cancelFailure); - } - } - } + maybeCancelSearchTask(); onShardGroupFailure(shardIndex, shard, e); } if (lastShard == false) { performPhaseOnShard(shardIndex, shardIt, nextShard); } else { - if ((int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1) { - onPhaseDone(); - } + finishShardAndMaybePhase(); } } @@ -385,40 +343,8 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, // temporary error. e = NoShardAvailableActionException.forOnShardFailureWrapper(e.getMessage()); } - // we don't aggregate shard on failures due to the internal cancellation, - // but do keep the header counts right - if ((requestCancelled.get() && isTaskCancelledException(e)) == false) { - AtomicArray shardFailures = this.shardFailures.get(); - // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures) - if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally - synchronized (shardFailuresMutex) { - shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it? - if (shardFailures == null) { // still null so we are the first and create a new instance - shardFailures = new AtomicArray<>(results.getNumShards()); - this.shardFailures.set(shardFailures); - } - } - } - ShardSearchFailure failure = shardFailures.get(shardIndex); - if (failure == null) { - shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); - } else { - // the failure is already present, try and not override it with an exception that is less meaningless - // for example, getting illegal shard state - if (TransportActions.isReadOverrideException(e) && (e instanceof SearchContextMissingException == false)) { - shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); - } - } - - if (results.hasResult(shardIndex)) { - assert failure == null : "shard failed before but shouldn't: " + failure; - successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter - } - } - } - static boolean isTaskCancelledException(Exception e) { - return ExceptionsHelper.unwrapCausesAndSuppressed(e, ex -> ex instanceof TaskCancelledException).isPresent(); + handleFailedAndCancelled(shardIndex, shardTarget, e); } /** @@ -446,16 +372,11 @@ private void onShardResultConsumed(Result result) { if (shardFailures != null) { shardFailures.set(result.getShardIndex(), null); } - // we need to increment successful ops first before we compare the exit condition otherwise if we - // are fast we could concurrently update totalOps but then preempt one of the threads which can - // cause the successor to read a wrong value from successfulOps if second phase is very fast ie. count etc. - // increment all the "future" shards to update the total ops since we some may work and some may not... - // and when that happens, we break on total ops, so we must maintain them - successfulShardExecution(); + finishShardAndMaybePhase(); } - private void successfulShardExecution() { - if ((int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1) { + private void finishShardAndMaybePhase() { + if (finishShard()) { onPhaseDone(); } } diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index fdce2541afcdd..4e0c60a075c22 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -18,6 +18,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.OriginalIndices; import org.elasticsearch.action.ShardOperationFailedException; +import org.elasticsearch.action.support.TransportActions; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.bytes.BytesReference; @@ -25,14 +26,19 @@ import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.core.Releasable; import org.elasticsearch.core.Releasables; +import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchShardTarget; import org.elasticsearch.search.builder.PointInTimeBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; +import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.search.internal.ShardSearchContextId; +import org.elasticsearch.tasks.TaskCancelledException; import org.elasticsearch.transport.Transport; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.VarHandle; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -84,6 +90,19 @@ public abstract class AsyncSearchContext { protected final TransportSearchAction.SearchTimeProvider timeProvider; protected final SearchResponse.Clusters clusters; + private static final VarHandle OUTSTANDING_SHARDS; + + static { + try { + OUTSTANDING_SHARDS = MethodHandles.lookup().findVarHandle(AsyncSearchContext.class, "outstandingShards", int.class); + } catch (Exception e) { + throw new ExceptionInInitializerError(e); + } + } + + @SuppressWarnings("unused") // only accessed via #OUTSTANDING_SHARDS + private int outstandingShards; + /** * Used by subclasses to resolve node ids to DiscoveryNodes. **/ @@ -123,6 +142,7 @@ protected AsyncSearchContext( // that participate in the search request. This means that this number is // consistent between two requests that target the same shards. Arrays.sort(shardIterators); + outstandingShards = shardIterators.length; this.request = request; this.results = results; this.namedWriteableRegistry = namedWriteableRegistry; @@ -156,6 +176,10 @@ protected void notifyListShards( ); } + static boolean isTaskCancelledException(Exception e) { + return ExceptionsHelper.unwrapCausesAndSuppressed(e, ex -> ex instanceof TaskCancelledException).isPresent(); + } + static ShardSearchFailure[] buildShardFailures(SetOnce> shardFailuresRef) { AtomicArray shardFailures = shardFailuresRef.get(); if (shardFailures == null) { @@ -178,6 +202,69 @@ static boolean isPartOfPIT(NamedWriteableRegistry namedWriteableRegistry, Search } } + protected void maybeCancelSearchTask() { + if (request.allowPartialSearchResults() == false) { + if (requestCancelled.compareAndSet(false, true)) { + try { + searchTransportService.cancelSearchTask( + task.getId(), + "partial results are not allowed and at least one shard has failed" + ); + } catch (Exception cancelFailure) { + logger.debug("Failed to cancel search request", cancelFailure); + } + } + } + } + + protected final void sendZeroShardsResponse() { + // no search shards to search on, bail with empty response + // (it happens with search across _all with no indices around and consistent with broadcast operations) + var source = request.source(); + int trackTotalHitsUpTo = source == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO + : source.trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO + : source.trackTotalHitsUpTo(); + // total hits is null in the response if the tracking of total hits is disabled + boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; + sendSearchResponse( + withTotalHits ? SearchResponseSections.EMPTY_WITH_TOTAL_HITS : SearchResponseSections.EMPTY_WITHOUT_TOTAL_HITS, + new AtomicArray<>(0) + ); + } + + protected final void handleFailedAndCancelled(int shardIndex, SearchShardTarget shardTarget, Exception e) { + // we don't aggregate shard on failures due to the internal cancellation, + // but do keep the header counts right + if ((requestCancelled.get() && isTaskCancelledException(e)) == false) { + AtomicArray shardFailures = this.shardFailures.get(); + // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures) + if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally + synchronized (shardFailuresMutex) { + shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it? + if (shardFailures == null) { // still null so we are the first and create a new instance + shardFailures = new AtomicArray<>(results.getNumShards()); + this.shardFailures.set(shardFailures); + } + } + } + ShardSearchFailure failure = shardFailures.get(shardIndex); + if (failure == null) { + shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); + } else { + // the failure is already present, try and not override it with an exception that is less meaningless + // for example, getting illegal shard state + if (TransportActions.isReadOverrideException(e) && (e instanceof SearchContextMissingException == false)) { + shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); + } + } + + if (results.hasResult(shardIndex)) { + assert (int) OUTSTANDING_SHARDS.getAcquire(this) == 0 : "should only be called by subsequent phases, not during query"; + assert failure == null : "shard failed before but shouldn't: " + failure; + successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we + return (int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1; + } + /** * Returns the currently executing search request */ diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 108f079e182e3..096b0dafe9269 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -44,7 +44,6 @@ import org.elasticsearch.core.SimpleRefCounted; import org.elasticsearch.core.TimeValue; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.search.SearchContextMissingException; import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.SearchShardTarget; @@ -67,8 +66,6 @@ import org.elasticsearch.transport.TransportResponseHandler; import java.io.IOException; -import java.lang.invoke.MethodHandles; -import java.lang.invoke.VarHandle; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -95,19 +92,6 @@ public class SearchQueryThenFetchAsyncAction e private static final Logger logger = LogManager.getLogger(SearchQueryThenFetchAsyncAction.class); - private static final VarHandle OUTSTANDING_SHARDS; - - static { - try { - OUTSTANDING_SHARDS = MethodHandles.lookup() - .findVarHandle(SearchQueryThenFetchAsyncAction.class, "outstandingShards", int.class); - } catch (Exception e) { - throw new ExceptionInInitializerError(e); - } - } - - private int outstandingShards; - private final SearchProgressListener progressListener; // informations to track the best bottom top doc globally. @@ -149,7 +133,6 @@ public class SearchQueryThenFetchAsyncAction e clusterState, clusters ); - outstandingShards = shardIterators.length; this.topDocsSize = getTopDocsSize(request); this.trackTotalHitsUpTo = request.resolveTrackTotalHitsUpTo(); this.client = client; @@ -166,18 +149,7 @@ public class SearchQueryThenFetchAsyncAction e */ public final void start() { if (results.getNumShards() == 0) { - // no search shards to search on, bail with empty response - // (it happens with search across _all with no indices around and consistent with broadcast operations) - var source = request.source(); - int trackTotalHitsUpTo = source == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO - : source.trackTotalHitsUpTo() == null ? SearchContext.DEFAULT_TRACK_TOTAL_HITS_UP_TO - : source.trackTotalHitsUpTo(); - // total hits is null in the response if the tracking of total hits is disabled - boolean withTotalHits = trackTotalHitsUpTo != SearchContext.TRACK_TOTAL_HITS_DISABLED; - sendSearchResponse( - withTotalHits ? SearchResponseSections.EMPTY_WITH_TOTAL_HITS : SearchResponseSections.EMPTY_WITHOUT_TOTAL_HITS, - new AtomicArray<>(0) - ); + sendZeroShardsResponse(); return; } try { @@ -515,16 +487,16 @@ private static ShardSearchRequest rewriteShardSearchRequest( } private void run() { + if (shardsIts.size() == 0) { + finish(); + return; + } // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); for (int i = 0; i < shardIterators.length; i++) { var iterator = shardIterators[i]; shardIndexMap.put(iterator, i); } - if (shardsIts.size() == 0) { - finish(); - return; - } final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); AbstractSearchAsyncAction.doCheckNoMissingShards(NAME, request, shardsIts, AbstractSearchAsyncAction::makeMissingShardsError); @@ -707,18 +679,7 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final final boolean lastShard = nextShard == null; logger.debug(() -> format("%s: Failed to execute [%s] lastShard [%s]", shard, request, lastShard), e); if (lastShard) { - if (request.allowPartialSearchResults() == false) { - if (requestCancelled.compareAndSet(false, true)) { - try { - searchTransportService.cancelSearchTask( - task.getId(), - "partial results are not allowed and at least one shard has failed" - ); - } catch (Exception cancelFailure) { - logger.debug("Failed to cancel search request", cancelFailure); - } - } - } + maybeCancelSearchTask(); onShardGroupFailure(shardIndex, shard, e); finishShardAndMaybePhase(); } else { @@ -727,7 +688,7 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } private void finishShardAndMaybePhase() { - if ((int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1) { + if (finishShard()) { finish(); } } @@ -750,37 +711,7 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, // temporary error. e = NoShardAvailableActionException.forOnShardFailureWrapper(e.getMessage()); } - // we don't aggregate shard on failures due to the internal cancellation, - // but do keep the header counts right - if ((requestCancelled.get() && AbstractSearchAsyncAction.isTaskCancelledException(e)) == false) { - AtomicArray shardFailures = this.shardFailures.get(); - // lazily create shard failures, so we can early build the empty shard failure list in most cases (no failures) - if (shardFailures == null) { // this is double checked locking but it's fine since SetOnce uses a volatile read internally - synchronized (shardFailuresMutex) { - shardFailures = this.shardFailures.get(); // read again otherwise somebody else has created it? - if (shardFailures == null) { // still null so we are the first and create a new instance - shardFailures = new AtomicArray<>(results.getNumShards()); - this.shardFailures.set(shardFailures); - } - } - } - ShardSearchFailure failure = shardFailures.get(shardIndex); - if (failure == null) { - shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); - } else { - // the failure is already present, try and not override it with an exception that is less meaningless - // for example, getting illegal shard state - if (TransportActions.isReadOverrideException(e) && (e instanceof SearchContextMissingException == false)) { - shardFailures.set(shardIndex, new ShardSearchFailure(e, shardTarget)); - } - } - - if (results.hasResult(shardIndex)) { - assert (int) OUTSTANDING_SHARDS.getAcquire(this) == 0 : "should only be called by subsequent phases, not during query"; - assert failure == null : "shard failed before but shouldn't: " + failure; - successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we have to adjust the counter - } - } + handleFailedAndCancelled(shardIndex, shardTarget, e); } @Override From 5548744c00ef03aa1cc03e7659a4dd12b8e55a23 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Sun, 26 Jan 2025 16:24:51 +0100 Subject: [PATCH 111/132] fixed --- .../elasticsearch/action/search/AsyncSearchContext.java | 9 +++++++-- .../action/search/AbstractSearchAsyncActionTests.java | 4 ++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 4e0c60a075c22..c5a5a4de7f63c 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -153,7 +153,7 @@ protected AsyncSearchContext( this.nodeIdToConnection = nodeIdToConnection; // register the release of the query consumer to free up the circuit breaker memory // at the end of the search - addReleasable(results); + releasables.add(results); this.timeProvider = timeProvider; this.concreteIndexBoosts = concreteIndexBoosts; @@ -261,7 +261,12 @@ protected final void handleFailedAndCancelled(int shardIndex, SearchShardTarget if (results.hasResult(shardIndex)) { assert (int) OUTSTANDING_SHARDS.getAcquire(this) == 0 : "should only be called by subsequent phases, not during query"; assert failure == null : "shard failed before but shouldn't: " + failure; - successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we + successfulOps.decrementAndGet(); // if this shard was successful before (initial phase) we need to count down the successes + } + } + } + + protected final boolean finishShard() { return (int) OUTSTANDING_SHARDS.getAndAdd(this, -1) == 1; } diff --git a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java index 1ac8ab0d516e8..a15b0be6b4d8a 100644 --- a/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/AbstractSearchAsyncActionTests.java @@ -93,6 +93,10 @@ private AbstractSearchAsyncAction createAction( request.getMaxConcurrentShardRequests(), SearchResponse.Clusters.EMPTY ) { + { + assertTrue(finishShard()); // only have a single shard in the iterator, lets finish that one as is expected by tests + } + @Override protected SearchPhase getNextPhase() { return null; From ab6459952775ab441326fa3c8a2c56c8d2090320 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 28 Jan 2025 15:04:19 +0100 Subject: [PATCH 112/132] fix --- .../java/org/elasticsearch/action/search/AsyncSearchContext.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index c5a5a4de7f63c..523577234cd5a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -341,6 +341,7 @@ protected final void raisePhaseFailure(SearchPhaseExecutionException exception) } } }); + OUTSTANDING_SHARDS.setVolatile(this, 0); // we're done no more shards to process, the phase has failed listener.onFailure(exception); } From 8dec6ee1fe7be71e4952b9a0b33510a46bfa64d9 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 28 Jan 2025 16:29:25 +0100 Subject: [PATCH 113/132] short and clean --- .../SearchQueryThenFetchAsyncAction.java | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 096b0dafe9269..4eea9d7c3c14e 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -831,7 +831,8 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo } - private static void maybeRelease(SearchService searchService, NodeQueryRequest request, SearchPhaseResult phaseResult) { + private static void maybeRelease(SearchService searchService, NodeQueryRequest request, SearchPhaseResult result) { + var phaseResult = result.queryResult() != null ? result.queryResult() : result.rankFeatureResult(); if (phaseResult != null && phaseResult.hasSearchContext() && request.searchRequest.scroll() == null @@ -987,20 +988,8 @@ private QueryPerNodeState( try (queryPhaseResultConsumer) { var failure = queryPhaseResultConsumer.failure.get(); if (failure != null) { - try { - queryPhaseResultConsumer.getSuccessfulResults() - .forEach( - searchPhaseResult -> maybeRelease( - dependencies.searchService, - searchRequest, - searchPhaseResult.queryResult() != null - ? searchPhaseResult.queryResult() - : searchPhaseResult.rankFeatureResult() - ) - ); - } catch (Throwable e) { - throw new RuntimeException(e); - } + queryPhaseResultConsumer.getSuccessfulResults() + .forEach(searchPhaseResult -> maybeRelease(dependencies.searchService, searchRequest, searchPhaseResult)); channelListener.onFailure(failure); return; } From 381f90509d8294603261f731da94145679ffbf0d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 28 Jan 2025 17:25:51 +0100 Subject: [PATCH 114/132] short and clean --- .../action/search/SearchQueryThenFetchAsyncAction.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 4eea9d7c3c14e..74dab502a9205 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -1040,9 +1040,6 @@ private QueryPerNodeState( channelListener, new NodeQueryResponse(mergeResult, results, queryPhaseResultConsumer.topDocsStats) ); - queryPhaseResultConsumer.buffer = null; - } catch (Throwable e) { - throw new AssertionError(e); } } }; From 028e6cab3d857ae273762398eca72a7702a4abba Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 28 Jan 2025 17:33:35 +0100 Subject: [PATCH 115/132] fixes --- .../action/search/SearchQueryThenFetchAsyncAction.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 74dab502a9205..4aed5512a140f 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -849,17 +849,18 @@ protected void doRun() { var searchRequest = request.searchRequest; var pitBuilder = searchRequest.pointInTimeBuilder(); var shardToQuery = request.shards.get(dataNodeLocalIdx); + final var shardId = shardToQuery.shardId; state.dependencies.searchService.executeQueryPhase( rewriteShardSearchRequest( state.bottomSortCollector, state.trackTotalHitsUpTo, buildShardSearchRequest( - shardToQuery.shardId, + shardId, searchRequest.getLocalClusterAlias(), shardToQuery.shardIndex, shardToQuery.contextId, shardToQuery.originalIndices, - request.aliasFilters.getOrDefault(shardToQuery.shardId.getIndex().getUUID(), AliasFilter.EMPTY), + request.aliasFilters.getOrDefault(shardId.getIndex().getUUID(), AliasFilter.EMPTY), pitBuilder == null ? null : pitBuilder.getKeepAlive(), shardToQuery.boost, searchRequest, From 8374ee00e298cbc001e46a6927f10c7d43a1e1c4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 28 Jan 2025 17:41:39 +0100 Subject: [PATCH 116/132] cleanup --- .../action/search/SearchQueryThenFetchAsyncAction.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 4aed5512a140f..00584d73c1e14 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -1055,10 +1055,12 @@ void consumeResult(QuerySearchResult queryResult) { && queryResult.topDocs() != null && queryResult.topDocs().topDocs.getClass() == TopFieldDocs.class) { TopFieldDocs topDocs = (TopFieldDocs) queryResult.topDocs().topDocs; + var bottomSortCollector = this.bottomSortCollector; if (bottomSortCollector == null) { synchronized (this) { + bottomSortCollector = this.bottomSortCollector; if (bottomSortCollector == null) { - bottomSortCollector = new BottomSortValuesCollector(topDocsSize, topDocs.fields); + bottomSortCollector = this.bottomSortCollector = new BottomSortValuesCollector(topDocsSize, topDocs.fields); } } } From 1429c7c782ab7ca8f6fa99507eaca4d859fcc122 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 28 Jan 2025 18:34:57 +0100 Subject: [PATCH 117/132] cleanup --- .../search/search-your-data/search-across-clusters.asciidoc | 4 ---- .../java/org/elasticsearch/action/search/SearchResponse.java | 4 +--- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/docs/reference/search/search-your-data/search-across-clusters.asciidoc b/docs/reference/search/search-your-data/search-across-clusters.asciidoc index 377dddef1ae51..9bd320b91b0d3 100644 --- a/docs/reference/search/search-your-data/search-across-clusters.asciidoc +++ b/docs/reference/search/search-your-data/search-across-clusters.asciidoc @@ -260,7 +260,6 @@ The API returns the following response: { "took": 150, "timed_out": false, - "num_reduce_phases": 4, "_shards": { "total": 28, "successful": 28, @@ -439,7 +438,6 @@ The API returns the following response: "response": { "took": 1020, "timed_out": false, - "num_reduce_phases": 0, "_shards": { "total": 10, <2> "successful": 0, @@ -622,7 +620,6 @@ Response: "response": { "took": 27619, "timed_out": false, - "num_reduce_phases": 4, "_shards": { "total": 28, "successful": 28, <2> @@ -759,7 +756,6 @@ Response: "response": { "took": 2069, "timed_out": false, - "num_reduce_phases": 4, "_shards": { "total": 28, "successful": 27, diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java b/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java index 787dc14f6cd96..22cd5252fb066 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -417,9 +417,7 @@ public XContentBuilder headerToXContent(XContentBuilder builder, ToXContent.Para if (isTerminatedEarly() != null) { builder.field(TERMINATED_EARLY.getPreferredName(), isTerminatedEarly()); } - if (getNumReducePhases() != 1) { - builder.field(NUM_REDUCE_PHASES.getPreferredName(), getNumReducePhases()); - } + // TODO: bring back rendering reduce phase count RestActions.buildBroadcastShardsHeader( builder, params, From eac260da228345fb2e48ed613cd507dd62de370c Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 29 Jan 2025 02:20:50 +0100 Subject: [PATCH 118/132] cleanup --- .../rest-api-spec/test/search/120_batch_reduce_size.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml index c36d2580ec22e..4b2307ef79bc1 100644 --- a/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml +++ b/rest-api-spec/src/yamlRestTest/resources/rest-api-spec/test/search/120_batch_reduce_size.yml @@ -48,7 +48,6 @@ setup: batched_reduce_size: 2 body: { "size" : 0, "aggs" : { "str_terms" : { "terms" : { "field" : "str" } } } } - - match: { num_reduce_phases: 3 } - match: { hits.total: 3 } - length: { aggregations.str_terms.buckets: 2 } - match: { aggregations.str_terms.buckets.0.key: "abc" } From f1c01402de903e3176a5c153ebebe1734596bc19 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 29 Jan 2025 12:08:08 +0100 Subject: [PATCH 119/132] cleanup --- .../xpack/search/AsyncSearchResponseTests.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java index 98513f611a5d8..48280e179c031 100644 --- a/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java +++ b/x-pack/plugin/async-search/src/test/java/org/elasticsearch/xpack/search/AsyncSearchResponseTests.java @@ -268,7 +268,6 @@ public void testToXContentWithSearchResponseAfterCompletion() throws IOException "response" : { "took" : %s, "timed_out" : false, - "num_reduce_phases" : 2, "_shards" : { "total" : 10, "successful" : 9, @@ -304,7 +303,6 @@ public void testToXContentWithSearchResponseAfterCompletion() throws IOException "response" : { "took" : %s, "timed_out" : false, - "num_reduce_phases" : 2, "_shards" : { "total" : 10, "successful" : 9, @@ -388,7 +386,6 @@ public void testToXContentWithCCSSearchResponseWhileRunning() throws IOException "response" : { "took" : %s, "timed_out" : false, - "num_reduce_phases" : 2, "_shards" : { "total" : 10, "successful" : 9, @@ -447,7 +444,6 @@ public void testToXContentWithCCSSearchResponseWhileRunning() throws IOException "response" : { "took" : %s, "timed_out" : false, - "num_reduce_phases" : 2, "_shards" : { "total" : 10, "successful" : 9, @@ -623,7 +619,6 @@ public void testToXContentWithCCSSearchResponseAfterCompletion() throws IOExcept "response" : { "took" : %s, "timed_out" : true, - "num_reduce_phases" : 2, "_shards" : { "total" : 10, "successful" : 9, @@ -770,7 +765,6 @@ public void testToXContentWithSearchResponseWhileRunning() throws IOException { "response" : { "took" : %s, "timed_out" : false, - "num_reduce_phases" : 2, "_shards" : { "total" : 10, "successful" : 9, @@ -804,7 +798,6 @@ public void testToXContentWithSearchResponseWhileRunning() throws IOException { "response" : { "took" : %s, "timed_out" : false, - "num_reduce_phases" : 2, "_shards" : { "total" : 10, "successful" : 9, From 650f6b9f649a9f4b9a04904565051952dfc5aadf Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 29 Jan 2025 13:19:16 +0100 Subject: [PATCH 120/132] cleanup --- .../org/elasticsearch/search/CCSDuelIT.java | 28 ------------------- 1 file changed, 28 deletions(-) diff --git a/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java b/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java index 79cdc1047aec9..d7f87e574dee8 100644 --- a/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java +++ b/qa/multi-cluster-search/src/test/java/org/elasticsearch/search/CCSDuelIT.java @@ -1030,20 +1030,6 @@ private static Map duelSearchSync(SearchRequest searchRequest, C } ObjectPath minimizeRoundtripsSearchResponse = ObjectPath.createFromResponse(minimizeRoundtripsResponse.get()); responseChecker.accept(minimizeRoundtripsSearchResponse); - - // if only the remote cluster was searched, then only one reduce phase is expected - int expectedReducePhasesMinRoundTrip = 1; - if (searchRequest.indices().length > 1) { - expectedReducePhasesMinRoundTrip = searchRequest.indices().length + 1; - } - if (expectedReducePhasesMinRoundTrip == 1) { - assertThat( - minimizeRoundtripsSearchResponse.evaluate("num_reduce_phases"), - anyOf(equalTo(expectedReducePhasesMinRoundTrip), nullValue()) - ); - } else { - assertThat(minimizeRoundtripsSearchResponse.evaluate("num_reduce_phases"), equalTo(expectedReducePhasesMinRoundTrip)); - } ObjectPath fanOutSearchResponse = ObjectPath.createFromResponse(fanOutResponse.get()); responseChecker.accept(fanOutSearchResponse); assertThat(fanOutSearchResponse.evaluate("num_reduce_phases"), anyOf(equalTo(1), nullValue())); // default value is 1? @@ -1159,20 +1145,6 @@ private static Map duelSearchAsync( responseChecker.accept(minimizeRoundtripsResponse); - // if only the remote cluster was searched, then only one reduce phase is expected - int expectedReducePhasesMinRoundTrip = 1; - if (searchRequest.indices().length > 1) { - expectedReducePhasesMinRoundTrip = searchRequest.indices().length + 1; - } - if (expectedReducePhasesMinRoundTrip == 1) { - assertThat( - minimizeRoundtripsResponse.evaluate("num_reduce_phases"), - anyOf(equalTo(expectedReducePhasesMinRoundTrip), nullValue()) - ); - } else { - assertThat(minimizeRoundtripsResponse.evaluate("num_reduce_phases"), equalTo(expectedReducePhasesMinRoundTrip)); - } - responseChecker.accept(fanOutResponse); assertThat(fanOutResponse.evaluate("num_reduce_phases"), anyOf(equalTo(1), nullValue())); // default value is 1? From 5fb7d59c2e04bf1892768c472c0f59c33a6b227d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 29 Jan 2025 23:05:05 +0100 Subject: [PATCH 121/132] align more --- .../search/AbstractSearchAsyncAction.java | 44 ++----------------- .../action/search/AsyncSearchContext.java | 42 ++++++++++++++++++ .../SearchQueryThenFetchAsyncAction.java | 33 ++------------ 3 files changed, 49 insertions(+), 70 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index f38aef6fe3a01..7ddb949fc4562 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -177,7 +177,7 @@ public final void start() { private void run() { if (shardsIts.size() == 0) { - onPhaseDone(); + finish(); return; } final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); @@ -347,40 +347,6 @@ public void onShardFailure(final int shardIndex, SearchShardTarget shardTarget, handleFailedAndCancelled(shardIndex, shardTarget, e); } - /** - * Executed once for every successful shard level request. - * @param result the result returned form the shard - */ - protected void onShardResult(Result result) { - assert result.getShardIndex() != -1 : "shard index is not set"; - assert result.getSearchShardTarget() != null : "search shard target must not be null"; - if (hasShardResponse == false) { - hasShardResponse = true; - } - if (logger.isTraceEnabled()) { - logger.trace("got first-phase result from {}", result.getSearchShardTarget()); - } - results.consumeResult(result, () -> onShardResultConsumed(result)); - } - - private void onShardResultConsumed(Result result) { - successfulOps.incrementAndGet(); - // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level - // so its ok concurrency wise to miss potentially the shard failures being created because of another failure - // in the #addShardFailure, because by definition, it will happen on *another* shardIndex - AtomicArray shardFailures = this.shardFailures.get(); - if (shardFailures != null) { - shardFailures.set(result.getShardIndex(), null); - } - finishShardAndMaybePhase(); - } - - private void finishShardAndMaybePhase() { - if (finishShard()) { - onPhaseDone(); - } - } - private SearchResponse buildSearchResponse( SearchResponseSections internalSearchResponse, ShardSearchFailure[] failures, @@ -459,12 +425,8 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C } } - /** - * Executed once all shard results have been received and processed - * @see #onShardFailure(int, SearchShardTarget, Exception) - * @see #onShardResult(SearchPhaseResult) - */ - private void onPhaseDone() { // as a tribute to @kimchy aka. finishHim() + @Override + protected void finish() { // as a tribute to @kimchy aka. finishHim() executeNextPhase(name, this::getNextPhase); } diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 523577234cd5a..e9424a599e8e9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -398,4 +398,46 @@ protected BytesReference buildSearchContextId() { ? source.pointInTimeBuilder().getEncodedId() : null; } + + /** + * Executed once for every successful shard level request. + * @param result the result returned form the shard + */ + protected void onShardResult(Result result) { + assert result.getShardIndex() != -1 : "shard index is not set"; + assert result.getSearchShardTarget() != null : "search shard target must not be null"; + if (hasShardResponse == false) { + hasShardResponse = true; + } + if (logger.isTraceEnabled()) { + logger.trace("got first-phase result from {}", result.getSearchShardTarget()); + } + results.consumeResult(result, () -> onShardResultConsumed(result)); + } + + /** + * Executed once all shard results have been received and processed + * @see #onShardFailure(int, SearchShardTarget, Exception) + * @see #onShardResult(SearchPhaseResult) + */ + protected abstract void finish(); + + protected void finishShardAndMaybePhase() { + if (finishShard()) { + finish(); + } + } + + private void onShardResultConsumed(Result result) { + successfulOps.incrementAndGet(); + // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level + // so its ok concurrency wise to miss potentially the shard failures being created because of another failure + // in the #addShardFailure, because by definition, it will happen on *another* shardIndex + AtomicArray shardFailures = this.shardFailures.get(); + if (shardFailures != null) { + shardFailures.set(result.getShardIndex(), null); + } + finishShardAndMaybePhase(); + } + } diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 00584d73c1e14..058a6cb3c7bb0 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -418,6 +418,7 @@ protected void onShardGroupFailure(int shardIndex, SearchShardTarget shardTarget progressListener.notifyQueryFailure(shardIndex, shardTarget, exc); } + @Override protected void onShardResult(Result result) { QuerySearchResult queryResult = result.queryResult(); if (queryResult.isNull() == false @@ -437,15 +438,7 @@ protected void onShardResult(Result result) { } bottomSortCollector.consumeTopDocs(topDocs, queryResult.sortValueFormats()); } - assert result.getShardIndex() != -1 : "shard index is not set"; - assert result.getSearchShardTarget() != null : "search shard target must not be null"; - if (hasShardResponse == false) { - hasShardResponse = true; - } - if (logger.isTraceEnabled()) { - logger.trace("got first-phase result from {}", result != null ? result.getSearchShardTarget() : null); - } - results.consumeResult(result, () -> onShardResultConsumed(result)); + super.onShardResult(result); } static SearchPhase nextPhase( @@ -488,7 +481,7 @@ private static ShardSearchRequest rewriteShardSearchRequest( private void run() { if (shardsIts.size() == 0) { - finish(); + executeNextPhase(NAME, this::getNextPhase); return; } // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging @@ -687,13 +680,7 @@ private void onShardFailure(final int shardIndex, SearchShardTarget shard, final } } - private void finishShardAndMaybePhase() { - if (finishShard()) { - finish(); - } - } - - private void finish() { + protected void finish() { executeNextPhase(NAME, this::getNextPhase); } @@ -774,18 +761,6 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C } } - private void onShardResultConsumed(SearchPhaseResult result) { - successfulOps.incrementAndGet(); - // clean a previous error on this shard group (note, this code will be serialized on the same shardIndex value level - // so its ok concurrency wise to miss potentially the shard failures being created because of another failure - // in the #addShardFailure, because by definition, it will happen on *another* shardIndex - AtomicArray shardFailures = this.shardFailures.get(); - if (shardFailures != null) { - shardFailures.set(result.getShardIndex(), null); - } - finishShardAndMaybePhase(); - } - public static final String NODE_SEARCH_ACTION_NAME = "indices:data/read/search[query][n]"; private static final CircuitBreaker NOOP_CIRCUIT_BREAKER = new NoopCircuitBreaker("request"); From 7475d65d961f0e4cc7bbd76911b4f55f8e5e6b00 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Tue, 4 Feb 2025 15:29:16 +0100 Subject: [PATCH 122/132] cleanup --- .../search/CanMatchPreFilterSearchPhase.java | 356 +++++++++--------- .../TransportOpenPointInTimeAction.java | 4 +- .../action/search/TransportSearchAction.java | 12 +- .../search/TransportSearchShardsAction.java | 4 +- .../search/CanMatchShardResponse.java | 4 +- .../CanMatchPreFilterSearchPhaseTests.java | 31 +- 6 files changed, 216 insertions(+), 195 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java index 80444da37d827..7aa3f69840fcc 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java +++ b/server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java @@ -10,6 +10,7 @@ package org.elasticsearch.action.search; import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.FixedBitSet; import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.GroupShardsIterator; import org.elasticsearch.common.util.Maps; @@ -40,6 +41,7 @@ import java.util.concurrent.Executor; import java.util.concurrent.atomic.AtomicReferenceArray; import java.util.function.BiFunction; +import java.util.stream.Collectors; import static org.elasticsearch.core.Strings.format; import static org.elasticsearch.core.Types.forciblyCast; @@ -57,9 +59,6 @@ */ final class CanMatchPreFilterSearchPhase { - /** Value to use in {@link #res} for shards that match but where no {@link MinAndMax} value is available*/ - private static final Object TRUE_SENTINEL = new Object(); - private final Logger logger; private final SearchRequest request; private final GroupShardsIterator shardsIts; @@ -74,11 +73,12 @@ final class CanMatchPreFilterSearchPhase { private final Executor executor; private final boolean requireAtLeastOneMatch; + private final FixedBitSet possibleMatches; + private final MinAndMax[] minAndMaxes; + private int numPossibleMatches; private final CoordinatorRewriteContextProvider coordinatorRewriteContextProvider; - private final AtomicReferenceArray res; - - private CanMatchPreFilterSearchPhase( + CanMatchPreFilterSearchPhase( Logger logger, SearchTransportService searchTransportService, BiFunction nodeIdToConnection, @@ -106,9 +106,9 @@ private CanMatchPreFilterSearchPhase( this.requireAtLeastOneMatch = requireAtLeastOneMatch; this.coordinatorRewriteContextProvider = coordinatorRewriteContextProvider; this.executor = executor; - int size = shardsIts.size(); - res = new AtomicReferenceArray<>(size); - + final int size = shardsIts.size(); + possibleMatches = new FixedBitSet(size); + minAndMaxes = new MinAndMax[size]; // we compute the shard index based on the natural order of the shards // that participate in the search request. This means that this number is // consistent between two requests that target the same shards. @@ -125,42 +125,6 @@ private CanMatchPreFilterSearchPhase( this.shardItIndexMap = shardItIndexMap; } - public static void execute( - Logger logger, - SearchTransportService searchTransportService, - BiFunction nodeIdToConnection, - Map aliasFilter, - Map concreteIndexBoosts, - Executor executor, - SearchRequest request, - GroupShardsIterator shardsIts, - TransportSearchAction.SearchTimeProvider timeProvider, - SearchTask task, - boolean requireAtLeastOneMatch, - CoordinatorRewriteContextProvider coordinatorRewriteContextProvider, - ActionListener> listener - ) { - if (shardsIts.size() == 0) { - listener.onResponse(shardsIts); - return; - } - new CanMatchPreFilterSearchPhase( - logger, - searchTransportService, - nodeIdToConnection, - aliasFilter, - concreteIndexBoosts, - executor, - request, - shardsIts, - timeProvider, - task, - requireAtLeastOneMatch, - coordinatorRewriteContextProvider, - listener - ).start(); - } - private static boolean assertSearchCoordinationThread() { return ThreadPool.assertCurrentThreadPool(ThreadPool.Names.SEARCH_COORDINATION); } @@ -172,35 +136,34 @@ private void runCoordinatorRewritePhase() { assert assertSearchCoordinationThread(); final List matchedShardLevelRequests = new ArrayList<>(); for (SearchShardIterator searchShardIterator : shardsIts) { + final CanMatchNodeRequest canMatchNodeRequest = new CanMatchNodeRequest( + request, + searchShardIterator.getOriginalIndices().indicesOptions(), + Collections.emptyList(), + shardsIts.size(), + timeProvider.absoluteStartMillis(), + searchShardIterator.getClusterAlias() + ); + final ShardSearchRequest request = canMatchNodeRequest.createShardSearchRequest(buildShardLevelRequest(searchShardIterator)); if (searchShardIterator.prefiltered()) { - if (searchShardIterator.skip() == false) { - res.set(shardItIndexMap.get(searchShardIterator), TRUE_SENTINEL); - } + consumeResult(searchShardIterator.skip() == false, request); continue; } boolean canMatch = true; CoordinatorRewriteContext coordinatorRewriteContext = coordinatorRewriteContextProvider.getCoordinatorRewriteContext( - searchShardIterator.shardId().getIndex() + request.shardId().getIndex() ); if (coordinatorRewriteContext != null) { try { - canMatch = SearchService.queryStillMatchesAfterRewrite( - new CanMatchNodeRequest( - request, - searchShardIterator.getOriginalIndices().indicesOptions(), - Collections.emptyList(), - shardsIts.size(), - timeProvider.absoluteStartMillis(), - searchShardIterator.getClusterAlias() - ).createShardSearchRequest(buildShardLevelRequest(searchShardIterator)), - coordinatorRewriteContext - ); + canMatch = SearchService.queryStillMatchesAfterRewrite(request, coordinatorRewriteContext); } catch (Exception e) { // treat as if shard is still a potential match } } if (canMatch) { matchedShardLevelRequests.add(searchShardIterator); + } else { + consumeResult(false, request); } } if (matchedShardLevelRequests.isEmpty()) { @@ -208,100 +171,145 @@ private void runCoordinatorRewritePhase() { } else { GroupShardsIterator matchingShards = new GroupShardsIterator<>(matchedShardLevelRequests); // verify missing shards only for the shards that we hit for the query - AbstractSearchAsyncAction.doCheckNoMissingShards( - "can_match", - request, - matchingShards, - AbstractSearchAsyncAction::makeMissingShardsError - ); + checkNoMissingShards(matchingShards); new Round(matchingShards).run(); } } + private void consumeResult(boolean canMatch, ShardSearchRequest request) { + CanMatchShardResponse result = new CanMatchShardResponse(canMatch, null); + result.setShardIndex(request.shardRequestIndex()); + consumeResult(result, () -> {}); + } + + private void consumeResult(CanMatchShardResponse result, Runnable next) { + try { + final boolean canMatch = result.canMatch(); + final MinAndMax minAndMax = result.estimatedMinAndMax(); + if (canMatch || minAndMax != null) { + consumeResult(result.getShardIndex(), canMatch, minAndMax); + } + } finally { + next.run(); + } + } + + private synchronized void consumeResult(int shardIndex, boolean canMatch, MinAndMax minAndMax) { + if (canMatch) { + possibleMatches.set(shardIndex); + numPossibleMatches++; + } + minAndMaxes[shardIndex] = minAndMax; + } + + private void checkNoMissingShards(GroupShardsIterator shards) { + assert assertSearchCoordinationThread(); + AbstractSearchAsyncAction.doCheckNoMissingShards("can_match", request, shards, AbstractSearchAsyncAction::makeMissingShardsError); + } + + private Map> groupByNode(GroupShardsIterator shards) { + Map> requests = new HashMap<>(); + for (int i = 0; i < shards.size(); i++) { + final SearchShardIterator shardRoutings = shards.get(i); + assert shardRoutings.skip() == false; + assert shardItIndexMap.containsKey(shardRoutings); + SearchShardTarget target = shardRoutings.nextOrNull(); + if (target != null) { + requests.computeIfAbsent(new SendingTarget(target.getClusterAlias(), target.getNodeId()), t -> new ArrayList<>()) + .add(shardRoutings); + } else { + requests.computeIfAbsent(new SendingTarget(null, null), t -> new ArrayList<>()).add(shardRoutings); + } + } + return requests; + } + /** * Sending can-match requests is round-based and grouped per target node. * If there are failures during a round, there will be a follow-up round * to retry on other available shard copies. */ - private final class Round extends AbstractRunnable { + class Round extends AbstractRunnable { private final GroupShardsIterator shards; private final CountDown countDown; + private final AtomicReferenceArray failedResponses; Round(GroupShardsIterator shards) { this.shards = shards; this.countDown = new CountDown(shards.size()); + this.failedResponses = new AtomicReferenceArray<>(shardsIts.size()); } @Override protected void doRun() { assert assertSearchCoordinationThread(); - Map> requests = new HashMap<>(); - for (int i = 0; i < shards.size(); i++) { - final SearchShardIterator shardRoutings = shards.get(i); - assert shardRoutings.skip() == false; - assert shardItIndexMap.containsKey(shardRoutings); - SearchShardTarget target = shardRoutings.nextOrNull(); - if (target != null) { - requests.computeIfAbsent(new SendingTarget(target.getClusterAlias(), target.getNodeId()), t -> new ArrayList<>()) - .add(shardRoutings); - } else { - onOperationResult(shardItIndexMap.get(shardRoutings), TRUE_SENTINEL); - } - } + final Map> requests = groupByNode(shards); for (Map.Entry> entry : requests.entrySet()) { - CanMatchNodeRequest canMatchNodeRequest = createCanMatchRequest(entry.getValue()); + CanMatchNodeRequest canMatchNodeRequest = createCanMatchRequest(entry); List shardLevelRequests = canMatchNodeRequest.getShardLevelRequests(); - var sendingTarget = entry.getKey(); - var listener = new ActionListener() { - @Override - public void onResponse(CanMatchNodeResponse canMatchNodeResponse) { - var responses = canMatchNodeResponse.getResponses(); - assert responses.size() == shardLevelRequests.size(); - for (int i = 0; i < responses.size(); i++) { - CanMatchNodeResponse.ResponseOrFailure response = responses.get(i); - CanMatchShardResponse shardResponse = response.getResponse(); - final Object shardResult; - if (shardResponse != null) { - shardResult = shardResponse.canMatch() - ? Objects.requireNonNullElse(shardResponse.estimatedMinAndMax(), TRUE_SENTINEL) - : null; - } else { - Exception failure = response.getException(); - assert failure != null; - shardResult = failure; - } - onOperationResult(shardLevelRequests.get(i).getShardRequestIndex(), shardResult); - } - } - @Override - public void onFailure(Exception e) { - for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { - onOperationResult(shard.getShardRequestIndex(), e); - } + if (entry.getKey().nodeId == null) { + // no target node: just mark the requests as failed + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), null); } - }; + continue; + } + + var sendingTarget = entry.getKey(); try { searchTransportService.sendCanMatch( nodeIdToConnection.apply(sendingTarget.clusterAlias, sendingTarget.nodeId), canMatchNodeRequest, task, - listener + new ActionListener<>() { + @Override + public void onResponse(CanMatchNodeResponse canMatchNodeResponse) { + assert canMatchNodeResponse.getResponses().size() == canMatchNodeRequest.getShardLevelRequests().size(); + for (int i = 0; i < canMatchNodeResponse.getResponses().size(); i++) { + CanMatchNodeResponse.ResponseOrFailure response = canMatchNodeResponse.getResponses().get(i); + if (response.getResponse() != null) { + CanMatchShardResponse shardResponse = response.getResponse(); + shardResponse.setShardIndex(shardLevelRequests.get(i).getShardRequestIndex()); + onOperation(shardResponse.getShardIndex(), shardResponse); + } else { + Exception failure = response.getException(); + assert failure != null; + onOperationFailed(shardLevelRequests.get(i).getShardRequestIndex(), failure); + } + } + } + + @Override + public void onFailure(Exception e) { + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), e); + } + } + } ); } catch (Exception e) { - listener.onFailure(e); + for (CanMatchNodeRequest.Shard shard : shardLevelRequests) { + onOperationFailed(shard.getShardRequestIndex(), e); + } } } } - @Override - public boolean isForceExecution() { - return true; + private void onOperation(int idx, CanMatchShardResponse response) { + failedResponses.set(idx, null); + consumeResult(response, () -> { + if (countDown.countDown()) { + finishRound(); + } + }); } - private void onOperationResult(int idx, Object result) { - res.set(idx, result); + private void onOperationFailed(int idx, Exception e) { + failedResponses.set(idx, e); + // we have to carry over shard failures in order to account for them in the response. + consumeResult(idx, true, null); if (countDown.countDown()) { finishRound(); } @@ -310,8 +318,8 @@ private void onOperationResult(int idx, Object result) { private void finishRound() { List remainingShards = new ArrayList<>(); for (SearchShardIterator ssi : shards) { - int idx = shardItIndexMap.get(ssi); - Exception failedResponse = res.get(idx) instanceof Exception e ? e : null; + int shardIndex = shardItIndexMap.get(ssi); + Exception failedResponse = failedResponses.get(shardIndex); if (failedResponse != null) { remainingShards.add(ssi); } @@ -320,30 +328,41 @@ private void finishRound() { finishPhase(); } else { // trigger another round, forcing execution - executor.execute(new Round(new GroupShardsIterator<>(remainingShards))); + executor.execute(new Round(new GroupShardsIterator<>(remainingShards)) { + @Override + public boolean isForceExecution() { + return true; + } + }); } } @Override public void onFailure(Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while running [can_match] phase", request), e); + } onPhaseFailure("round", e); } } private record SendingTarget(@Nullable String clusterAlias, @Nullable String nodeId) {} - private CanMatchNodeRequest createCanMatchRequest(List iters) { - final SearchShardIterator first = iters.get(0); - assert iters.stream() - .allMatch( - ssi -> ssi != null - && Objects.equals(ssi.getOriginalIndices().indicesOptions(), first.getOriginalIndices().indicesOptions()) - && Objects.equals(ssi.getClusterAlias(), first.getClusterAlias()) - ); + private CanMatchNodeRequest createCanMatchRequest(Map.Entry> entry) { + final SearchShardIterator first = entry.getValue().get(0); + final List shardLevelRequests = entry.getValue() + .stream() + .map(this::buildShardLevelRequest) + .collect(Collectors.toCollection(ArrayList::new)); + assert entry.getValue().stream().allMatch(Objects::nonNull); + assert entry.getValue() + .stream() + .allMatch(ssi -> Objects.equals(ssi.getOriginalIndices().indicesOptions(), first.getOriginalIndices().indicesOptions())); + assert entry.getValue().stream().allMatch(ssi -> Objects.equals(ssi.getClusterAlias(), first.getClusterAlias())); return new CanMatchNodeRequest( request, first.getOriginalIndices().indicesOptions(), - iters.stream().map(this::buildShardLevelRequest).toList(), + shardLevelRequests, shardsIts.size(), timeProvider.absoluteStartMillis(), first.getClusterAlias() @@ -351,7 +370,7 @@ private CanMatchNodeRequest createCanMatchRequest(List iter } private void finishPhase() { - listener.onResponse(getIterator()); + listener.onResponse(getIterator(shardsIts)); } private static final float DEFAULT_INDEX_BOOST = 1.0f; @@ -373,11 +392,18 @@ public CanMatchNodeRequest.Shard buildShardLevelRequest(SearchShardIterator shar ); } - private void start() { + public void start() { + if (shardsIts.size() == 0) { + finishPhase(); + return; + } // Note that the search is failed when this task is rejected by the executor executor.execute(new AbstractRunnable() { @Override public void onFailure(Exception e) { + if (logger.isDebugEnabled()) { + logger.debug(() -> format("Failed to execute [%s] while running [can_match] phase", request), e); + } onPhaseFailure("start", e); } @@ -390,71 +416,58 @@ protected void doRun() { } private void onPhaseFailure(String msg, Exception cause) { - if (logger.isDebugEnabled()) { - logger.debug(() -> format("Failed to execute [%s] while running [can_match] phase", request), cause); - } listener.onFailure(new SearchPhaseExecutionException("can_match", msg, cause, ShardSearchFailure.EMPTY_ARRAY)); } - private GroupShardsIterator getIterator() { + private synchronized GroupShardsIterator getIterator(GroupShardsIterator shardsIts) { // TODO: pick the local shard when possible - if (requireAtLeastOneMatch) { - boolean isEmpty = true; - for (int i = 0, n = res.length(); i < n; i++) { - if (res.get(i) != null) { - isEmpty = false; + if (requireAtLeastOneMatch && numPossibleMatches == 0) { + // this is a special case where we have no hit but we need to get at least one search response in order + // to produce a valid search result with all the aggs etc. + // Since it's possible that some of the shards that we're skipping are + // unavailable, we would try to query the node that at least has some + // shards available in order to produce a valid search result. + int shardIndexToQuery = 0; + for (int i = 0; i < shardsIts.size(); i++) { + SearchShardIterator it = shardsIts.get(i); + if (it.size() > 0) { + shardIndexToQuery = i; + it.skip(false); // un-skip which is needed when all the remote shards were skipped by the remote can_match break; } } - if (isEmpty) { - // this is a special case where we have no hit, but we need to get at least one search response in order - // to produce a valid search result with all the aggs etc. - // Since it's possible that some of the shards that we're skipping are - // unavailable, we would try to query the node that at least has some - // shards available in order to produce a valid search result. - int shardIndexToQuery = 0; - for (int i = 0; i < shardsIts.size(); i++) { - SearchShardIterator it = shardsIts.get(i); - if (it.size() > 0) { - shardIndexToQuery = i; - it.skip(false); // un-skip which is needed when all the remote shards were skipped by the remote can_match - break; - } - } - res.set(shardIndexToQuery, TRUE_SENTINEL); - } + possibleMatches.set(shardIndexToQuery); } int i = 0; for (SearchShardIterator iter : shardsIts) { iter.reset(); - if (res.get(i++) != null) { + boolean match = possibleMatches.get(i++); + if (match) { assert iter.skip() == false; } else { iter.skip(true); } } - return shouldSortShards(res) - ? new GroupShardsIterator<>(sortShards(shardsIts, res, FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()).order())) - : shardsIts; - } - - private static MinAndMax getMinAndMax(AtomicReferenceArray res, int shardIndex) { - return res.get(shardIndex) instanceof MinAndMax m ? m : null; + if (shouldSortShards(minAndMaxes) == false) { + return shardsIts; + } + FieldSortBuilder fieldSort = FieldSortBuilder.getPrimaryFieldSortOrNull(request.source()); + return new GroupShardsIterator<>(sortShards(shardsIts, minAndMaxes, fieldSort.order())); } private static List sortShards( GroupShardsIterator shardsIts, - AtomicReferenceArray results, + MinAndMax[] minAndMaxes, SortOrder order ) { int bound = shardsIts.size(); - Integer[] toSort = new Integer[bound]; + List toSort = new ArrayList<>(bound); for (int i = 0; i < bound; i++) { - toSort[i] = i; + toSort.add(i); } Comparator> keyComparator = forciblyCast(MinAndMax.getComparator(order)); - Arrays.sort(toSort, (idx1, idx2) -> { - int res = keyComparator.compare(getMinAndMax(results, idx1), getMinAndMax(results, idx2)); + toSort.sort((idx1, idx2) -> { + int res = keyComparator.compare(minAndMaxes[idx1], minAndMaxes[idx2]); if (res != 0) { return res; } @@ -467,10 +480,9 @@ private static List sortShards( return list; } - private static boolean shouldSortShards(AtomicReferenceArray results) { + private static boolean shouldSortShards(MinAndMax[] minAndMaxes) { Class clazz = null; - for (int i = 0, n = results.length(); i < n; i++) { - var minAndMax = getMinAndMax(results, i); + for (MinAndMax minAndMax : minAndMaxes) { if (clazz == null) { clazz = minAndMax == null ? null : minAndMax.getMin().getClass(); } else if (minAndMax != null && clazz != minAndMax.getMin().getClass()) { diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java index 1f07ccfabbef6..2ebda40cee4ba 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportOpenPointInTimeAction.java @@ -164,7 +164,7 @@ public void runNewSearchPhase( // that is signaled to the local can match through the SearchShardIterator#prefiltered flag. Local shards do need to go // through the local can match phase. if (SearchService.canRewriteToMatchNone(searchRequest.source())) { - CanMatchPreFilterSearchPhase.execute( + new CanMatchPreFilterSearchPhase( logger, searchTransportService, connectionLookup, @@ -191,7 +191,7 @@ public void runNewSearchPhase( clusters ) ) - ); + ).start(); } else { runOpenPointInTimePhase( task, diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 3b34c560acf40..9e3a9925ca7ab 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -1476,7 +1476,7 @@ public void runNewSearchPhase( if (preFilter) { // only for aggs we need to contact shards even if there are no matches boolean requireAtLeastOneMatch = searchRequest.source() != null && searchRequest.source().aggregations() != null; - CanMatchPreFilterSearchPhase.execute( + new CanMatchPreFilterSearchPhase( logger, searchTransportService, connectionLookup, @@ -1489,8 +1489,8 @@ public void runNewSearchPhase( task, requireAtLeastOneMatch, searchService.getCoordinatorRewriteContextProvider(timeProvider::absoluteStartMillis), - listener.delegateFailureAndWrap( - (l, iters) -> runNewSearchPhase( + listener.delegateFailureAndWrap((l, iters) -> { + runNewSearchPhase( task, searchRequest, executor, @@ -1503,9 +1503,9 @@ public void runNewSearchPhase( false, threadPool, clusters - ) - ) - ); + ); + }) + ).start(); return; } // for synchronous CCS minimize_roundtrips=false, use the CCSSingleCoordinatorSearchProgressListener diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java index ed41845343104..614a3e9cf22ae 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchShardsAction.java @@ -150,7 +150,7 @@ public void searchShards(Task task, SearchShardsRequest searchShardsRequest, Act if (SearchService.canRewriteToMatchNone(searchRequest.source()) == false) { delegate.onResponse(new SearchShardsResponse(toGroups(shardIts), clusterState.nodes().getAllNodes(), aliasFilters)); } else { - CanMatchPreFilterSearchPhase.execute(logger, searchTransportService, (clusterAlias, node) -> { + new CanMatchPreFilterSearchPhase(logger, searchTransportService, (clusterAlias, node) -> { assert Objects.equals(clusterAlias, searchShardsRequest.clusterAlias()); return transportService.getConnection(clusterState.nodes().get(node)); }, @@ -164,7 +164,7 @@ public void searchShards(Task task, SearchShardsRequest searchShardsRequest, Act false, searchService.getCoordinatorRewriteContextProvider(timeProvider::absoluteStartMillis), delegate.map(its -> new SearchShardsResponse(toGroups(its), clusterState.nodes().getAllNodes(), aliasFilters)) - ); + ).start(); } }) ); diff --git a/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java index cb6983c90292a..985e7a52aae81 100644 --- a/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java +++ b/server/src/main/java/org/elasticsearch/search/CanMatchShardResponse.java @@ -11,7 +11,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.search.sort.MinAndMax; import java.io.IOException; @@ -19,11 +18,12 @@ /** * Shard-level response for can-match requests */ -public final class CanMatchShardResponse implements Writeable { +public final class CanMatchShardResponse extends SearchPhaseResult { private final boolean canMatch; private final MinAndMax estimatedMinAndMax; public CanMatchShardResponse(StreamInput in) throws IOException { + super(in); this.canMatch = in.readBoolean(); estimatedMinAndMax = in.readOptionalWriteable(MinAndMax::new); } diff --git a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java index e95685fe84453..1460270c48293 100644 --- a/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java +++ b/server/src/test/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhaseTests.java @@ -148,7 +148,7 @@ public void sendCanMatch( final SearchRequest searchRequest = new SearchRequest(); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase.execute( + CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -166,6 +166,8 @@ public void sendCanMatch( latch.countDown(); }) ); + + canMatchPhase.start(); latch.await(); assertThat(numRequests.get(), replicaNode == null ? equalTo(1) : lessThanOrEqualTo(2)); @@ -244,7 +246,7 @@ public void sendCanMatch( final SearchRequest searchRequest = new SearchRequest(); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase.execute( + CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -262,6 +264,8 @@ public void sendCanMatch( latch.countDown(); }) ); + + canMatchPhase.start(); latch.await(); assertEquals(0, result.get().get(0).shardId().id()); @@ -304,8 +308,8 @@ public void sendCanMatch( for (CanMatchNodeRequest.Shard shard : request.getShardLevelRequests()) { Long min = rarely() ? null : randomLong(); Long max = min == null ? null : randomLongBetween(min, Long.MAX_VALUE); + MinAndMax minMax = min == null ? null : new MinAndMax<>(min, max); boolean canMatch = frequently(); - MinAndMax minMax = canMatch == false || min == null ? null : new MinAndMax<>(min, max); synchronized (shardIds) { shardIds.add(shard.shardId()); minAndMaxes.add(minMax); @@ -335,7 +339,7 @@ public void sendCanMatch( searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase.execute( + CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -353,6 +357,8 @@ public void sendCanMatch( latch.countDown(); }) ); + + canMatchPhase.start(); latch.await(); ShardId[] expected = IntStream.range(0, shardIds.size()) .boxed() @@ -435,7 +441,7 @@ public void sendCanMatch( searchRequest.source(new SearchSourceBuilder().sort(SortBuilders.fieldSort("timestamp").order(order))); searchRequest.allowPartialSearchResults(true); - CanMatchPreFilterSearchPhase.execute( + CanMatchPreFilterSearchPhase canMatchPhase = new CanMatchPreFilterSearchPhase( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), @@ -453,6 +459,8 @@ public void sendCanMatch( latch.countDown(); }) ); + + canMatchPhase.start(); latch.await(); int shardId = 0; for (SearchShardIterator i : result.get()) { @@ -1183,7 +1191,7 @@ public void testCanMatchFilteringOnCoordinatorWithMissingShards() throws Excepti // test that a search does fail if the query does NOT filter ALL the // unassigned shards CountDownLatch latch = new CountDownLatch(1); - Tuple> canMatchPhaseAndRequests = getCanMatchPhaseAndRequests( + Tuple> canMatchPhaseAndRequests = getCanMatchPhaseAndRequests( List.of(dataStream), List.of(hotRegularIndex, warmRegularIndex), coordinatorRewriteContextProvider, @@ -1207,7 +1215,7 @@ public void onFailure(Exception e) { } ); - canMatchPhaseAndRequests.v1().run(); + canMatchPhaseAndRequests.v1().start(); latch.await(10, TimeUnit.SECONDS); } } @@ -1262,7 +1270,7 @@ private void assignShardsAndExecuteCanMatchPhase( ) throws Exception { AtomicReference> result = new AtomicReference<>(); CountDownLatch latch = new CountDownLatch(1); - Tuple> canMatchAndShardRequests = getCanMatchPhaseAndRequests( + Tuple> canMatchAndShardRequests = getCanMatchPhaseAndRequests( dataStreams, regularIndices, contextProvider, @@ -1277,7 +1285,7 @@ private void assignShardsAndExecuteCanMatchPhase( }) ); - canMatchAndShardRequests.v1().run(); + canMatchAndShardRequests.v1().start(); latch.await(); List updatedSearchShardIterators = new ArrayList<>(); @@ -1288,7 +1296,7 @@ private void assignShardsAndExecuteCanMatchPhase( canMatchResultsConsumer.accept(updatedSearchShardIterators, canMatchAndShardRequests.v2()); } - private Tuple> getCanMatchPhaseAndRequests( + private Tuple> getCanMatchPhaseAndRequests( List dataStreams, List regularIndices, CoordinatorRewriteContextProvider contextProvider, @@ -1407,8 +1415,9 @@ public void sendCanMatch( System::nanoTime ); + AtomicReference> result = new AtomicReference<>(); return new Tuple<>( - () -> CanMatchPreFilterSearchPhase.execute( + new CanMatchPreFilterSearchPhase( logger, searchTransportService, (clusterAlias, node) -> lookup.get(node), From c432cd8667a9f291c225a95be5b2ebc563f6ee9b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 5 Feb 2025 21:59:23 +0100 Subject: [PATCH 123/132] drier --- .../action/search/AbstractSearchAsyncAction.java | 4 ---- .../action/search/AsyncSearchContext.java | 10 ++++++---- .../action/search/SearchQueryThenFetchAsyncAction.java | 5 ----- 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java index ca9a032069b19..baf7ca836ea12 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java @@ -430,10 +430,6 @@ protected void finish() { // as a tribute to @kimchy aka. finishHim() executeNextPhase(name, this::getNextPhase); } - public final void execute(Runnable command) { - executor.execute(command); - } - /** * Builds an request for the initial search phase. * diff --git a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java index 57f7e6e042f9f..b6fbec39dbe84 100644 --- a/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java +++ b/server/src/main/java/org/elasticsearch/action/search/AsyncSearchContext.java @@ -65,21 +65,21 @@ public abstract class AsyncSearchContext { // protected for tests protected final List releasables = new ArrayList<>(); - protected final AtomicBoolean requestCancelled = new AtomicBoolean(); + private final AtomicBoolean requestCancelled = new AtomicBoolean(); protected final SearchTask task; protected final AtomicInteger successfulOps = new AtomicInteger(); protected final SearchTransportService searchTransportService; - protected final Executor executor; + private final Executor executor; protected final List toSkipShardsIts; protected final List shardsIts; protected final SearchShardIterator[] shardIterators; protected final SetOnce> shardFailures = new SetOnce<>(); - protected final Object shardFailuresMutex = new Object(); + private final Object shardFailuresMutex = new Object(); protected final TransportVersion minTransportVersion; protected final Map aliasFilter; @@ -288,7 +288,9 @@ public final void addReleasable(Releasable releasable) { releasables.add(releasable); } - abstract void execute(Runnable command); + public final void execute(Runnable command) { + executor.execute(command); + } abstract void onShardFailure(int shardIndex, SearchShardTarget shard, Exception e); diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index a3756b57ecd3f..9f923fd7f38ab 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -747,11 +747,6 @@ public void onPhaseFailure(String phase, String msg, Throwable cause) { raisePhaseFailure(new SearchPhaseExecutionException(phase, msg, cause, buildShardFailures(shardFailures))); } - @Override - public void execute(Runnable command) { - executor.execute(command); - } - @Override public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.Connection connection) { assert isPartOfPointInTime(contextId) == false : "Must not release point in time context [" + contextId + "]"; From 5b2028b2812a704e744c48d16ab2e79a65a56e00 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 5 Feb 2025 22:29:11 +0100 Subject: [PATCH 124/132] missing optimization --- .../SearchQueryThenFetchAsyncAction.java | 26 ++++++++++++++----- 1 file changed, 19 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 9f923fd7f38ab..39b430c509b54 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -126,7 +126,9 @@ public class SearchQueryThenFetchAsyncAction e executor, nodeIdToConnection, shardsIts, - aliasFilter, + removeEmptyAliasFilters(aliasFilter), // TODO: we should do this at build time for this map, but BwC is tricky for now + // this is an important optimization for batched execution though because it keeps + // the transport request size in check by avoiding redundant index name/uuid strings concreteIndexBoosts, timeProvider, clusterState, @@ -152,6 +154,10 @@ public final void start() { return; } try { + if (shardsIts.isEmpty()) { + executeNextPhase(NAME, this::getNextPhase); + return; + } run(); } catch (Exception e) { if (logger.isDebugEnabled()) { @@ -267,8 +273,8 @@ public static class NodeQueryResponse extends TransportResponse { SearchPhaseController.TopDocsStats topDocsStats ) { this.results = results; - for (int i = 0; i < results.length; i++) { - if (results[i] instanceof RefCounted r) { + for (Object result : results) { + if (result instanceof RefCounted r) { r.incRef(); } } @@ -479,10 +485,6 @@ private static ShardSearchRequest rewriteShardSearchRequest( } private void run() { - if (shardsIts.size() == 0) { - executeNextPhase(NAME, this::getNextPhase); - return; - } // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); for (int i = 0; i < shardIterators.length; i++) { @@ -594,6 +596,16 @@ public void handleException(TransportException e) { }); } + private static Map removeEmptyAliasFilters(Map aliasFilters) { + Map aliasFilterNoEmpty = new HashMap<>(); + aliasFilters.forEach((idx, filter) -> { + if (AliasFilter.EMPTY.equals(filter) == false) { + aliasFilterNoEmpty.put(idx, filter); + } + }); + return Map.copyOf(aliasFilterNoEmpty); + } + private void onNodeQueryFailure(Exception e, NodeQueryRequest request, String nodeId) { for (ShardToQuery shard : request.shards) { int idx = shard.shardIndex; From fa3244508cbdf11a37fe0c97a3368432adc1fc8b Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 5 Feb 2025 22:55:59 +0100 Subject: [PATCH 125/132] cleanups --- .../SearchQueryThenFetchAsyncAction.java | 146 +++++++++--------- 1 file changed, 70 insertions(+), 76 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 39b430c509b54..0a8ac5bd229aa 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -488,8 +488,7 @@ private void run() { // TODO: stupid but we kinda need to fill all of these in with the current logic, do something nicer before merging final Map shardIndexMap = Maps.newHashMapWithExpectedSize(shardIterators.length); for (int i = 0; i < shardIterators.length; i++) { - var iterator = shardIterators[i]; - shardIndexMap.put(iterator, i); + shardIndexMap.put(shardIterators[i], i); } final boolean supportsBatchedQuery = minTransportVersion.onOrAfter(TransportVersions.BATCHED_QUERY_PHASE_VERSION); final Map perNodeQueries = new HashMap<>(); @@ -773,11 +772,9 @@ public void sendReleaseSearchContext(ShardSearchContextId contextId, Transport.C public static void registerNodeSearchAction(SearchTransportService searchTransportService, SearchService searchService) { var transportService = searchTransportService.transportService(); - final Dependencies dependencies = new Dependencies( - searchService, - transportService.getThreadPool().executor(ThreadPool.Names.SEARCH) - ); - final int searchPoolMax = transportService.getThreadPool().info(ThreadPool.Names.SEARCH).getMax(); + var threadPool = transportService.getThreadPool(); + final Dependencies dependencies = new Dependencies(searchService, threadPool.executor(ThreadPool.Names.SEARCH)); + final int searchPoolMax = threadPool.info(ThreadPool.Names.SEARCH).getMax(); final SearchPhaseController searchPhaseController = new SearchPhaseController(searchService::aggReduceContextBuilder); transportService.registerRequestHandler( NODE_SEARCH_ACTION_NAME, @@ -856,16 +853,13 @@ protected void doRun() { public void onResponse(SearchPhaseResult searchPhaseResult) { try { searchPhaseResult.setShardIndex(dataNodeLocalIdx); - final SearchShardTarget target = new SearchShardTarget( - null, - shardToQuery.shardId, - request.searchRequest.getLocalClusterAlias() + searchPhaseResult.setSearchShardTarget( + new SearchShardTarget(null, shardToQuery.shardId, request.searchRequest.getLocalClusterAlias()) ); - searchPhaseResult.setSearchShardTarget(target); // no need for any cache effects when we're already flipped to ture => plain read + set-release state.hasResponse.compareAndExchangeRelease(false, true); state.consumeResult(searchPhaseResult.queryResult()); - state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, state.onDone); + state.queryPhaseResultConsumer.consumeResult(searchPhaseResult, state::onDone); } catch (Exception e) { setFailure(state, dataNodeLocalIdx, e); } finally { @@ -875,7 +869,7 @@ public void onResponse(SearchPhaseResult searchPhaseResult) { private void setFailure(QueryPerNodeState state, int dataNodeLocalIdx, Exception e) { state.failures.put(dataNodeLocalIdx, e); - state.onDone.run(); + state.onDone(); } @Override @@ -904,7 +898,7 @@ public void onFailure(Exception e) { // TODO this could be done better now, we probably should only make sure to have a single loop running at // minimum and ignore + requeue rejections in that case state.failures.put(dataNodeLocalIdx, e); - state.onDone.run(); + state.onDone(); // TODO SO risk! maybeNext(); } @@ -941,10 +935,11 @@ private static final class QueryPerNodeState { private final CancellableTask task; private final ConcurrentHashMap failures = new ConcurrentHashMap<>(); private final Dependencies dependencies; - private final Runnable onDone; private final AtomicBoolean hasResponse = new AtomicBoolean(false); private final int trackTotalHitsUpTo; private final int topDocsSize; + private final CountDown countDown; + private final TransportChannel channel; private volatile BottomSortValuesCollector bottomSortCollector; private QueryPerNodeState( @@ -961,70 +956,69 @@ private QueryPerNodeState( this.trackTotalHitsUpTo = searchRequest.searchRequest.resolveTrackTotalHitsUpTo(); topDocsSize = getTopDocsSize(searchRequest.searchRequest); this.task = task; - final int shardCount = queryPhaseResultConsumer.getNumShards(); - final CountDown countDown = new CountDown(shardCount); + countDown = new CountDown(queryPhaseResultConsumer.getNumShards()); + this.channel = channel; this.dependencies = dependencies; - this.onDone = () -> { - if (countDown.countDown()) { - var channelListener = new ChannelActionListener<>(channel); - try (queryPhaseResultConsumer) { - var failure = queryPhaseResultConsumer.failure.get(); - if (failure != null) { - queryPhaseResultConsumer.getSuccessfulResults() - .forEach(searchPhaseResult -> maybeRelease(dependencies.searchService, searchRequest, searchPhaseResult)); - channelListener.onFailure(failure); - return; - } - final Object[] results = new Object[shardCount]; - for (int i = 0; i < results.length; i++) { - var e = failures.get(i); - var res = queryPhaseResultConsumer.results.get(i); - if (e != null) { - results[i] = e; - assert res == null; - } else { - results[i] = res; - assert results[i] != null; - } - } - final QueryPhaseResultConsumer.MergeResult mergeResult; - try { - mergeResult = Objects.requireNonNullElse( - queryPhaseResultConsumer.consumePartialResult(), - EMPTY_PARTIAL_MERGE_RESULT - ); - } catch (Exception e) { - channelListener.onFailure(e); - return; - } - // translate shard indices to those on the coordinator so that it can interpret the merge result without adjustments - final Set relevantShardIndices = new HashSet<>(); - for (ScoreDoc scoreDoc : mergeResult.reducedTopDocs().scoreDocs) { - final int localIndex = scoreDoc.shardIndex; - scoreDoc.shardIndex = searchRequest.shards.get(localIndex).shardIndex; - relevantShardIndices.add(localIndex); - } - for (Object result : results) { - if (result instanceof QuerySearchResult q - && q.getContextId() != null - && relevantShardIndices.contains(q.getShardIndex()) == false - && q.hasSuggestHits() == false - && q.getRankShardResult() == null - && searchRequest.searchRequest.scroll() == null - && (AsyncSearchContext.isPartOfPIT(null, searchRequest.searchRequest, q.getContextId()) == false)) { - if (dependencies.searchService.freeReaderContext(q.getContextId())) { - q.clearContextId(); - } - } - } + } - ActionListener.respondAndRelease( - channelListener, - new NodeQueryResponse(mergeResult, results, queryPhaseResultConsumer.topDocsStats) - ); + void onDone() { + if (countDown.countDown() == false) { + return; + } + var channelListener = new ChannelActionListener<>(channel); + try (queryPhaseResultConsumer) { + var failure = queryPhaseResultConsumer.failure.get(); + if (failure != null) { + queryPhaseResultConsumer.getSuccessfulResults() + .forEach(searchPhaseResult -> maybeRelease(dependencies.searchService, searchRequest, searchPhaseResult)); + channelListener.onFailure(failure); + return; + } + final Object[] results = new Object[queryPhaseResultConsumer.getNumShards()]; + for (int i = 0; i < results.length; i++) { + var e = failures.get(i); + var res = queryPhaseResultConsumer.results.get(i); + if (e != null) { + results[i] = e; + assert res == null; + } else { + results[i] = res; + assert results[i] != null; } } - }; + final QueryPhaseResultConsumer.MergeResult mergeResult; + try { + mergeResult = Objects.requireNonNullElse(queryPhaseResultConsumer.consumePartialResult(), EMPTY_PARTIAL_MERGE_RESULT); + } catch (Exception e) { + channelListener.onFailure(e); + return; + } + // translate shard indices to those on the coordinator so that it can interpret the merge result without adjustments + final Set relevantShardIndices = new HashSet<>(); + for (ScoreDoc scoreDoc : mergeResult.reducedTopDocs().scoreDocs) { + final int localIndex = scoreDoc.shardIndex; + scoreDoc.shardIndex = searchRequest.shards.get(localIndex).shardIndex; + relevantShardIndices.add(localIndex); + } + for (Object result : results) { + if (result instanceof QuerySearchResult q + && q.getContextId() != null + && relevantShardIndices.contains(q.getShardIndex()) == false + && q.hasSuggestHits() == false + && q.getRankShardResult() == null + && searchRequest.searchRequest.scroll() == null + && (AsyncSearchContext.isPartOfPIT(null, searchRequest.searchRequest, q.getContextId()) == false)) { + if (dependencies.searchService.freeReaderContext(q.getContextId())) { + q.clearContextId(); + } + } + } + + ActionListener.respondAndRelease( + channelListener, + new NodeQueryResponse(mergeResult, results, queryPhaseResultConsumer.topDocsStats) + ); + } } void consumeResult(QuerySearchResult queryResult) { From 4607032d15b878e811d4a12278525f72568e0e35 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 5 Feb 2025 23:06:30 +0100 Subject: [PATCH 126/132] revert test noise --- .../search/simple/SimpleSearchIT.java | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java b/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java index c6cf0618e7a8a..5a9be73d92268 100644 --- a/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java +++ b/server/src/internalClusterTest/java/org/elasticsearch/search/simple/SimpleSearchIT.java @@ -236,16 +236,14 @@ public void testRangeQueryKeyword() throws Exception { ensureGreen(); refresh(); - var coord = internalCluster().startCoordinatingOnlyNode(Settings.EMPTY); - var client = client(coord); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lte("B")), 2L); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("A").lte("B")), 1L); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lt("B")), 1L); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte(null).lt("C")), 3L); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("B").lt(null)), 2L); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt(null).lt(null)), 4L); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("").lt(null)), 4L); - assertHitCountAndNoFailures(client.prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("").lt(null)), 3L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lte("B")), 2L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("A").lte("B")), 1L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("A").lt("B")), 1L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte(null).lt("C")), 3L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("B").lt(null)), 2L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt(null).lt(null)), 4L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gte("").lt(null)), 4L); + assertHitCountAndNoFailures(prepareSearch("test").setQuery(QueryBuilders.rangeQuery("field").gt("").lt(null)), 3L); } public void testSimpleTerminateAfterCount() throws Exception { From 26d1b94c757d92df20ca4cfdbe487786eb1f9634 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 5 Feb 2025 23:37:12 +0100 Subject: [PATCH 127/132] cleanup --- .../java/org/elasticsearch/action/search/SearchRequest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java index dc9556d2af771..8b77ec7fb5463 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchRequest.java @@ -64,7 +64,7 @@ public class SearchRequest extends ActionRequest implements IndicesRequest.Repla private final String localClusterAlias; private final long absoluteStartMillis; - public boolean finalReduce; + private final boolean finalReduce; private SearchType searchType = SearchType.DEFAULT; From d30b2a5d5862e43c4d5cb257c5ee0b8dd5b7118d Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 6 Feb 2025 02:18:04 +0100 Subject: [PATCH 128/132] TODOs --- .../action/search/SearchQueryThenFetchAsyncAction.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java index 0a8ac5bd229aa..45df36847946a 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java @@ -793,7 +793,7 @@ public static void registerNodeSearchAction(SearchTransportService searchTranspo ((CancellableTask) task)::isCancelled, SearchProgressListener.NOOP, shardCount, - Integer.MAX_VALUE, + Integer.MAX_VALUE, // TODO: intermediary reduces e -> logger.error("failed to merge on data node", e) ), request, @@ -875,6 +875,7 @@ private void setFailure(QueryPerNodeState state, int dataNodeLocalIdx, Exception @Override public void onFailure(Exception e) { try { + // TODO: count down fully and just respond with an exception if partial results aren't allowed setFailure(state, dataNodeLocalIdx, e); maybeNext(); } catch (Throwable expected) { From bb2af54a6f5d5b0e8bb78e7f6aeb9e473d14a436 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 6 Feb 2025 02:23:22 +0100 Subject: [PATCH 129/132] revert debug change --- .../org/elasticsearch/action/ActionListener.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ActionListener.java b/server/src/main/java/org/elasticsearch/action/ActionListener.java index 79d0d29a591f1..a158669d936fe 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListener.java @@ -388,19 +388,14 @@ static ActionListener assertOnce(ActionListener d // if complete, records the stack trace which first completed it private final AtomicReference firstCompletion = new AtomicReference<>(); - private void assertFirstRun(Exception t) { + private void assertFirstRun() { var previousRun = firstCompletion.compareAndExchange(null, new ElasticsearchException("executed already")); - // reports the stack traces of both completions - assert previousRun == null - : new AssertionError( - "[" + delegate + "] " + previousRun, - t == null ? previousRun : ExceptionsHelper.useOrSuppress(previousRun, t) - ); + assert previousRun == null : "[" + delegate + "] " + previousRun; // reports the stack traces of both completions } @Override public void onResponse(Response response) { - assertFirstRun(null); + assertFirstRun(); try { delegate.onResponse(response); } catch (Exception e) { @@ -411,7 +406,7 @@ public void onResponse(Response response) { @Override public void onFailure(Exception e) { - assertFirstRun(e); + assertFirstRun(); safeOnFailure(delegate, e); } From aee2d1861519b647f174b89487fac9606e286a29 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 6 Feb 2025 02:44:34 +0100 Subject: [PATCH 130/132] revert debug change + dry --- .../action/search/ArraySearchPhaseResults.java | 2 +- .../org/elasticsearch/common/lucene/Lucene.java | 14 ++++---------- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java b/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java index 516d6c6b3eff1..e4d1f9f63bd2b 100644 --- a/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java +++ b/server/src/main/java/org/elasticsearch/action/search/ArraySearchPhaseResults.java @@ -43,7 +43,7 @@ Stream getSuccessfulResults() { @Override void consumeResult(Result result, Runnable next) { assert results.get(result.getShardIndex()) == null : "shardIndex: " + result.getShardIndex() + " is already set"; - results.setOnce(result.getShardIndex(), result); + results.set(result.getShardIndex(), result); result.incRef(); next.run(); } diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 59f3c4d8938b2..2639570470472 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -429,7 +429,10 @@ public static void writeTopDocsIncludingShardIndex(StreamOutput out, TopDocs top } else { out.writeByte((byte) 0); writeTotalHits(out, topDocs.totalHits); - out.writeArray(Lucene::writeScoreDocWithShardIndex, topDocs.scoreDocs); + out.writeArray((o, scoreDoc) -> { + writeScoreDoc(o, scoreDoc); + o.writeVInt(scoreDoc.shardIndex); + }, topDocs.scoreDocs); } } @@ -587,15 +590,6 @@ public static void writeScoreDoc(StreamOutput out, ScoreDoc scoreDoc) throws IOE out.writeFloat(scoreDoc.score); } - public static void writeScoreDocWithShardIndex(StreamOutput out, ScoreDoc scoreDoc) throws IOException { - if (scoreDoc.getClass().equals(ScoreDoc.class) == false) { - throw new IllegalArgumentException("This method can only be used to serialize a ScoreDoc, not a " + scoreDoc.getClass()); - } - out.writeVInt(scoreDoc.doc); - out.writeFloat(scoreDoc.score); - out.writeVInt(scoreDoc.shardIndex); - } - // LUCENE 4 UPGRADE: We might want to maintain our own ordinal, instead of Lucene's ordinal public static SortField.Type readSortType(StreamInput in) throws IOException { return SortField.Type.values()[in.readVInt()]; From 79e495ad5df297376c89f5615d7ce41fbddc31c4 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 6 Feb 2025 02:54:51 +0100 Subject: [PATCH 131/132] revert debug change + dry --- .../org/elasticsearch/common/lucene/Lucene.java | 2 +- .../org/elasticsearch/search/SearchService.java | 13 +++++-------- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java index 2639570470472..390eaa987a010 100644 --- a/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java +++ b/server/src/main/java/org/elasticsearch/common/lucene/Lucene.java @@ -391,7 +391,7 @@ public static ScoreDoc readScoreDoc(StreamInput in) throws IOException { } public static ScoreDoc readScoreDocWithShardIndex(StreamInput in) throws IOException { - var res = new ScoreDoc(in.readVInt(), in.readFloat()); + var res = readScoreDoc(in); res.shardIndex = in.readVInt(); return res; } diff --git a/server/src/main/java/org/elasticsearch/search/SearchService.java b/server/src/main/java/org/elasticsearch/search/SearchService.java index 93802b009b473..f47fc1b97456e 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchService.java +++ b/server/src/main/java/org/elasticsearch/search/SearchService.java @@ -322,8 +322,6 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv private final Tracer tracer; - private final CircuitBreaker circuitBreaker; - public SearchService( ClusterService clusterService, IndicesService indicesService, @@ -342,8 +340,11 @@ public SearchService( this.scriptService = scriptService; this.bigArrays = bigArrays; this.fetchPhase = fetchPhase; - this.circuitBreaker = circuitBreakerService.getBreaker(CircuitBreaker.REQUEST); - this.multiBucketConsumerService = new MultiBucketConsumerService(clusterService, settings, circuitBreaker); + this.multiBucketConsumerService = new MultiBucketConsumerService( + clusterService, + settings, + circuitBreakerService.getBreaker(CircuitBreaker.REQUEST) + ); this.executorSelector = executorSelector; this.tracer = tracer; @@ -390,10 +391,6 @@ public SearchService( .addSettingsUpdateConsumer(QUERY_PHASE_PARALLEL_COLLECTION_ENABLED, this::setEnableQueryPhaseParallelCollection); } - public CircuitBreaker circuitBreaker() { - return circuitBreaker; - } - private void setEnableSearchWorkerThreads(boolean enableSearchWorkerThreads) { if (enableSearchWorkerThreads) { searchExecutor = threadPool.executor(Names.SEARCH); From 2ee7640f6e35f957559d952baa2d316eda3b1e15 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 6 Feb 2025 03:02:14 +0100 Subject: [PATCH 132/132] cleanup --- .../action/search/CountOnlyQueryPhaseResultConsumer.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java b/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java index efbdf48e68ec9..b52d76aac4132 100644 --- a/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java +++ b/server/src/main/java/org/elasticsearch/action/search/CountOnlyQueryPhaseResultConsumer.java @@ -64,13 +64,6 @@ public void consumeResult(SearchPhaseResult result, Runnable next) { next.run(); } - public void reduce(boolean terminatedEarly, boolean timedOut, long totalHits, TotalHits.Relation totalHitsRelation) { - relationAtomicReference.compareAndSet(TotalHits.Relation.EQUAL_TO, totalHitsRelation); - this.totalHits.add(totalHits); - this.terminatedEarly.set(terminatedEarly); - this.timedOut.set(timedOut); - } - @Override boolean hasResult(int shardIndex) { return results.contains(shardIndex);