Skip to content

Commit 8b8c0c0

Browse files
authored
Add additional shards routing info in ShardSearchRequest (#29533)
This commit propagates the preference and routing of the original SearchRequest in the ShardSearchRequest. This information is then use to fix a bug in sliced scrolls when executed with a preference (or a routing). Instead of computing the slice query from the total number of shards in the index, this commit computes this number from the number of shards per index that participates in the request. Fixes #27550
1 parent fadcce8 commit 8b8c0c0

27 files changed

+605
-284
lines changed

server/src/main/java/org/elasticsearch/action/search/AbstractSearchAsyncAction.java

Lines changed: 17 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -37,8 +37,10 @@
3737
import org.elasticsearch.search.internal.ShardSearchTransportRequest;
3838
import org.elasticsearch.transport.Transport;
3939

40+
import java.util.Collections;
4041
import java.util.List;
4142
import java.util.Map;
43+
import java.util.Set;
4244
import java.util.concurrent.Executor;
4345
import java.util.concurrent.TimeUnit;
4446
import java.util.concurrent.atomic.AtomicInteger;
@@ -62,6 +64,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
6264
private final long clusterStateVersion;
6365
private final Map<String, AliasFilter> aliasFilter;
6466
private final Map<String, Float> concreteIndexBoosts;
67+
private final Map<String, Set<String>> indexRoutings;
6568
private final SetOnce<AtomicArray<ShardSearchFailure>> shardFailures = new SetOnce<>();
6669
private final Object shardFailuresMutex = new Object();
6770
private final AtomicInteger successfulOps = new AtomicInteger();
@@ -72,6 +75,7 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
7275
protected AbstractSearchAsyncAction(String name, Logger logger, SearchTransportService searchTransportService,
7376
BiFunction<String, String, Transport.Connection> nodeIdToConnection,
7477
Map<String, AliasFilter> aliasFilter, Map<String, Float> concreteIndexBoosts,
78+
Map<String, Set<String>> indexRoutings,
7579
Executor executor, SearchRequest request,
7680
ActionListener<SearchResponse> listener, GroupShardsIterator<SearchShardIterator> shardsIts,
7781
TransportSearchAction.SearchTimeProvider timeProvider, long clusterStateVersion,
@@ -89,6 +93,7 @@ protected AbstractSearchAsyncAction(String name, Logger logger, SearchTransportS
8993
this.clusterStateVersion = clusterStateVersion;
9094
this.concreteIndexBoosts = concreteIndexBoosts;
9195
this.aliasFilter = aliasFilter;
96+
this.indexRoutings = indexRoutings;
9297
this.results = resultConsumer;
9398
this.clusters = clusters;
9499
}
@@ -128,17 +133,17 @@ public final void executeNextPhase(SearchPhase currentPhase, SearchPhase nextPha
128133
onPhaseFailure(currentPhase, "all shards failed", cause);
129134
} else {
130135
Boolean allowPartialResults = request.allowPartialSearchResults();
131-
assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults";
136+
assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults";
132137
if (allowPartialResults == false && shardFailures.get() != null ){
133138
if (logger.isDebugEnabled()) {
134139
final ShardOperationFailedException[] shardSearchFailures = ExceptionsHelper.groupBy(buildShardFailures());
135140
Throwable cause = shardSearchFailures.length == 0 ? null :
136141
ElasticsearchException.guessRootCauses(shardSearchFailures[0].getCause())[0];
137-
logger.debug(() -> new ParameterizedMessage("{} shards failed for phase: [{}]",
142+
logger.debug(() -> new ParameterizedMessage("{} shards failed for phase: [{}]",
138143
shardSearchFailures.length, getName()), cause);
139144
}
140-
onPhaseFailure(currentPhase, "Partial shards failure", null);
141-
} else {
145+
onPhaseFailure(currentPhase, "Partial shards failure", null);
146+
} else {
142147
if (logger.isTraceEnabled()) {
143148
final String resultsFrom = results.getSuccessfulResults()
144149
.map(r -> r.getSearchShardTarget().toString()).collect(Collectors.joining(","));
@@ -271,14 +276,14 @@ public final SearchRequest getRequest() {
271276

272277
@Override
273278
public final SearchResponse buildSearchResponse(InternalSearchResponse internalSearchResponse, String scrollId) {
274-
279+
275280
ShardSearchFailure[] failures = buildShardFailures();
276281
Boolean allowPartialResults = request.allowPartialSearchResults();
277282
assert allowPartialResults != null : "SearchRequest missing setting for allowPartialSearchResults";
278283
if (allowPartialResults == false && failures.length > 0){
279-
raisePhaseFailure(new SearchPhaseExecutionException("", "Shard failures", null, failures));
280-
}
281-
284+
raisePhaseFailure(new SearchPhaseExecutionException("", "Shard failures", null, failures));
285+
}
286+
282287
return new SearchResponse(internalSearchResponse, scrollId, getNumShards(), successfulOps.get(),
283288
skippedOps.get(), buildTookInMillis(), failures, clusters);
284289
}
@@ -318,8 +323,11 @@ public final ShardSearchTransportRequest buildShardSearchRequest(SearchShardIter
318323
AliasFilter filter = aliasFilter.get(shardIt.shardId().getIndex().getUUID());
319324
assert filter != null;
320325
float indexBoost = concreteIndexBoosts.getOrDefault(shardIt.shardId().getIndex().getUUID(), DEFAULT_INDEX_BOOST);
326+
String indexName = shardIt.shardId().getIndex().getName();
327+
final String[] routings = indexRoutings.getOrDefault(indexName, Collections.emptySet())
328+
.toArray(new String[0]);
321329
return new ShardSearchTransportRequest(shardIt.getOriginalIndices(), request, shardIt.shardId(), getNumShards(),
322-
filter, indexBoost, timeProvider.getAbsoluteStartMillis(), clusterAlias);
330+
filter, indexBoost, timeProvider.getAbsoluteStartMillis(), clusterAlias, routings);
323331
}
324332

325333
/**

server/src/main/java/org/elasticsearch/action/search/CanMatchPreFilterSearchPhase.java

Lines changed: 5 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@
2727
import org.elasticsearch.transport.Transport;
2828

2929
import java.util.Map;
30+
import java.util.Set;
3031
import java.util.concurrent.Executor;
3132
import java.util.function.BiFunction;
3233
import java.util.function.Function;
@@ -47,6 +48,7 @@ final class CanMatchPreFilterSearchPhase extends AbstractSearchAsyncAction<Searc
4748
CanMatchPreFilterSearchPhase(Logger logger, SearchTransportService searchTransportService,
4849
BiFunction<String, String, Transport.Connection> nodeIdToConnection,
4950
Map<String, AliasFilter> aliasFilter, Map<String, Float> concreteIndexBoosts,
51+
Map<String, Set<String>> indexRoutings,
5052
Executor executor, SearchRequest request,
5153
ActionListener<SearchResponse> listener, GroupShardsIterator<SearchShardIterator> shardsIts,
5254
TransportSearchAction.SearchTimeProvider timeProvider, long clusterStateVersion,
@@ -56,9 +58,9 @@ final class CanMatchPreFilterSearchPhase extends AbstractSearchAsyncAction<Searc
5658
* We set max concurrent shard requests to the number of shards to otherwise avoid deep recursing that would occur if the local node
5759
* is the coordinating node for the query, holds all the shards for the request, and there are a lot of shards.
5860
*/
59-
super("can_match", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request,
60-
listener, shardsIts, timeProvider, clusterStateVersion, task, new BitSetSearchPhaseResults(shardsIts.size()), shardsIts.size(),
61-
clusters);
61+
super("can_match", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, indexRoutings,
62+
executor, request, listener, shardsIts, timeProvider, clusterStateVersion, task,
63+
new BitSetSearchPhaseResults(shardsIts.size()), shardsIts.size(), clusters);
6264
this.phaseFactory = phaseFactory;
6365
this.shardsIts = shardsIts;
6466
}

server/src/main/java/org/elasticsearch/action/search/InitialSearchPhase.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -131,7 +131,7 @@ public final void run() throws IOException {
131131
if (shardsIts.size() > 0) {
132132
int maxConcurrentShardRequests = Math.min(this.maxConcurrentShardRequests, shardsIts.size());
133133
final boolean success = shardExecutionIndex.compareAndSet(0, maxConcurrentShardRequests);
134-
assert success;
134+
assert success;
135135
assert request.allowPartialSearchResults() != null : "SearchRequest missing setting for allowPartialSearchResults";
136136
if (request.allowPartialSearchResults() == false) {
137137
final StringBuilder missingShards = new StringBuilder();
@@ -140,7 +140,7 @@ public final void run() throws IOException {
140140
final SearchShardIterator shardRoutings = shardsIts.get(index);
141141
if (shardRoutings.size() == 0) {
142142
if(missingShards.length() >0 ){
143-
missingShards.append(", ");
143+
missingShards.append(", ");
144144
}
145145
missingShards.append(shardRoutings.shardId());
146146
}

server/src/main/java/org/elasticsearch/action/search/SearchDfsQueryThenFetchAsyncAction.java

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import org.elasticsearch.transport.Transport;
2929

3030
import java.util.Map;
31+
import java.util.Set;
3132
import java.util.concurrent.Executor;
3233
import java.util.function.BiFunction;
3334

@@ -37,11 +38,13 @@ final class SearchDfsQueryThenFetchAsyncAction extends AbstractSearchAsyncAction
3738

3839
SearchDfsQueryThenFetchAsyncAction(final Logger logger, final SearchTransportService searchTransportService,
3940
final BiFunction<String, String, Transport.Connection> nodeIdToConnection, final Map<String, AliasFilter> aliasFilter,
40-
final Map<String, Float> concreteIndexBoosts, final SearchPhaseController searchPhaseController, final Executor executor,
41+
final Map<String, Float> concreteIndexBoosts, final Map<String, Set<String>> indexRoutings,
42+
final SearchPhaseController searchPhaseController, final Executor executor,
4143
final SearchRequest request, final ActionListener<SearchResponse> listener,
4244
final GroupShardsIterator<SearchShardIterator> shardsIts, final TransportSearchAction.SearchTimeProvider timeProvider,
4345
final long clusterStateVersion, final SearchTask task, SearchResponse.Clusters clusters) {
44-
super("dfs", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, listener,
46+
super("dfs", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, indexRoutings,
47+
executor, request, listener,
4548
shardsIts, timeProvider, clusterStateVersion, task, new ArraySearchPhaseResults<>(shardsIts.size()),
4649
request.getMaxConcurrentShardRequests(), clusters);
4750
this.searchPhaseController = searchPhaseController;

server/src/main/java/org/elasticsearch/action/search/SearchQueryThenFetchAsyncAction.java

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import org.elasticsearch.transport.Transport;
2929

3030
import java.util.Map;
31+
import java.util.Set;
3132
import java.util.concurrent.Executor;
3233
import java.util.function.BiFunction;
3334

@@ -37,13 +38,14 @@ final class SearchQueryThenFetchAsyncAction extends AbstractSearchAsyncAction<Se
3738

3839
SearchQueryThenFetchAsyncAction(final Logger logger, final SearchTransportService searchTransportService,
3940
final BiFunction<String, String, Transport.Connection> nodeIdToConnection, final Map<String, AliasFilter> aliasFilter,
40-
final Map<String, Float> concreteIndexBoosts, final SearchPhaseController searchPhaseController, final Executor executor,
41+
final Map<String, Float> concreteIndexBoosts, final Map<String, Set<String>> indexRoutings,
42+
final SearchPhaseController searchPhaseController, final Executor executor,
4143
final SearchRequest request, final ActionListener<SearchResponse> listener,
4244
final GroupShardsIterator<SearchShardIterator> shardsIts, final TransportSearchAction.SearchTimeProvider timeProvider,
4345
long clusterStateVersion, SearchTask task, SearchResponse.Clusters clusters) {
44-
super("query", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, executor, request, listener,
45-
shardsIts, timeProvider, clusterStateVersion, task, searchPhaseController.newSearchPhaseResults(request, shardsIts.size()),
46-
request.getMaxConcurrentShardRequests(), clusters);
46+
super("query", logger, searchTransportService, nodeIdToConnection, aliasFilter, concreteIndexBoosts, indexRoutings,
47+
executor, request, listener, shardsIts, timeProvider, clusterStateVersion, task,
48+
searchPhaseController.newSearchPhaseResults(request, shardsIts.size()), request.getMaxConcurrentShardRequests(), clusters);
4749
this.searchPhaseController = searchPhaseController;
4850
}
4951

server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java

Lines changed: 13 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -297,6 +297,7 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea
297297
Map<String, AliasFilter> aliasFilter = buildPerIndexAliasFilter(searchRequest, clusterState, indices, remoteAliasMap);
298298
Map<String, Set<String>> routingMap = indexNameExpressionResolver.resolveSearchRouting(clusterState, searchRequest.routing(),
299299
searchRequest.indices());
300+
routingMap = routingMap == null ? Collections.emptyMap() : Collections.unmodifiableMap(routingMap);
300301
String[] concreteIndices = new String[indices.length];
301302
for (int i = 0; i < indices.length; i++) {
302303
concreteIndices[i] = indices[i].getName();
@@ -350,7 +351,7 @@ private void executeSearch(SearchTask task, SearchTimeProvider timeProvider, Sea
350351
}
351352
boolean preFilterSearchShards = shouldPreFilterSearchShards(searchRequest, shardIterators);
352353
searchAsyncAction(task, searchRequest, shardIterators, timeProvider, connectionLookup, clusterState.version(),
353-
Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, listener, preFilterSearchShards, clusters).start();
354+
Collections.unmodifiableMap(aliasFilter), concreteIndexBoosts, routingMap, listener, preFilterSearchShards, clusters).start();
354355
}
355356

356357
private boolean shouldPreFilterSearchShards(SearchRequest searchRequest, GroupShardsIterator<SearchShardIterator> shardIterators) {
@@ -380,17 +381,20 @@ private AbstractSearchAsyncAction searchAsyncAction(SearchTask task, SearchReque
380381
GroupShardsIterator<SearchShardIterator> shardIterators,
381382
SearchTimeProvider timeProvider,
382383
BiFunction<String, String, Transport.Connection> connectionLookup,
383-
long clusterStateVersion, Map<String, AliasFilter> aliasFilter,
384+
long clusterStateVersion,
385+
Map<String, AliasFilter> aliasFilter,
384386
Map<String, Float> concreteIndexBoosts,
385-
ActionListener<SearchResponse> listener, boolean preFilter,
387+
Map<String, Set<String>> indexRoutings,
388+
ActionListener<SearchResponse> listener,
389+
boolean preFilter,
386390
SearchResponse.Clusters clusters) {
387391
Executor executor = threadPool.executor(ThreadPool.Names.SEARCH);
388392
if (preFilter) {
389393
return new CanMatchPreFilterSearchPhase(logger, searchTransportService, connectionLookup,
390-
aliasFilter, concreteIndexBoosts, executor, searchRequest, listener, shardIterators,
394+
aliasFilter, concreteIndexBoosts, indexRoutings, executor, searchRequest, listener, shardIterators,
391395
timeProvider, clusterStateVersion, task, (iter) -> {
392396
AbstractSearchAsyncAction action = searchAsyncAction(task, searchRequest, iter, timeProvider, connectionLookup,
393-
clusterStateVersion, aliasFilter, concreteIndexBoosts, listener, false, clusters);
397+
clusterStateVersion, aliasFilter, concreteIndexBoosts, indexRoutings, listener, false, clusters);
394398
return new SearchPhase(action.getName()) {
395399
@Override
396400
public void run() throws IOException {
@@ -403,14 +407,14 @@ public void run() throws IOException {
403407
switch (searchRequest.searchType()) {
404408
case DFS_QUERY_THEN_FETCH:
405409
searchAsyncAction = new SearchDfsQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup,
406-
aliasFilter, concreteIndexBoosts, searchPhaseController, executor, searchRequest, listener, shardIterators,
407-
timeProvider, clusterStateVersion, task, clusters);
410+
aliasFilter, concreteIndexBoosts, indexRoutings, searchPhaseController, executor, searchRequest, listener,
411+
shardIterators, timeProvider, clusterStateVersion, task, clusters);
408412
break;
409413
case QUERY_AND_FETCH:
410414
case QUERY_THEN_FETCH:
411415
searchAsyncAction = new SearchQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup,
412-
aliasFilter, concreteIndexBoosts, searchPhaseController, executor, searchRequest, listener, shardIterators,
413-
timeProvider, clusterStateVersion, task, clusters);
416+
aliasFilter, concreteIndexBoosts, indexRoutings, searchPhaseController, executor, searchRequest, listener,
417+
shardIterators, timeProvider, clusterStateVersion, task, clusters);
414418
break;
415419
default:
416420
throw new IllegalStateException("Unknown search type: [" + searchRequest.searchType() + "]");

server/src/main/java/org/elasticsearch/cluster/routing/PlainShardsIterator.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,7 @@
2424

2525
/**
2626
* A simple {@link ShardsIterator} that iterates a list or sub-list of
27-
* {@link ShardRouting shard routings}.
27+
* {@link ShardRouting shard indexRoutings}.
2828
*/
2929
public class PlainShardsIterator implements ShardsIterator {
3030

server/src/main/java/org/elasticsearch/cluster/routing/ShardRouting.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,7 @@
3838

3939
/**
4040
* {@link ShardRouting} immutably encapsulates information about shard
41-
* routings like id, state, version, etc.
41+
* indexRoutings like id, state, version, etc.
4242
*/
4343
public final class ShardRouting implements Writeable, ToXContentObject {
4444

@@ -477,7 +477,7 @@ public boolean isRelocationTargetOf(ShardRouting other) {
477477
"ShardRouting is a relocation target but current node id isn't equal to source relocating node. This [" + this + "], other [" + other + "]";
478478

479479
assert b == false || this.shardId.equals(other.shardId) :
480-
"ShardRouting is a relocation target but both routings are not of the same shard id. This [" + this + "], other [" + other + "]";
480+
"ShardRouting is a relocation target but both indexRoutings are not of the same shard id. This [" + this + "], other [" + other + "]";
481481

482482
assert b == false || this.primary == other.primary :
483483
"ShardRouting is a relocation target but primary flag is different. This [" + this + "], target [" + other + "]";
@@ -504,7 +504,7 @@ public boolean isRelocationSourceOf(ShardRouting other) {
504504
"ShardRouting is a relocation source but relocating node isn't equal to other's current node. This [" + this + "], other [" + other + "]";
505505

506506
assert b == false || this.shardId.equals(other.shardId) :
507-
"ShardRouting is a relocation source but both routings are not of the same shard. This [" + this + "], target [" + other + "]";
507+
"ShardRouting is a relocation source but both indexRoutings are not of the same shard. This [" + this + "], target [" + other + "]";
508508

509509
assert b == false || this.primary == other.primary :
510510
"ShardRouting is a relocation source but primary flag is different. This [" + this + "], target [" + other + "]";

0 commit comments

Comments
 (0)