33
33
import org .elasticsearch .search .SearchPhaseResult ;
34
34
import org .elasticsearch .search .SearchShardTarget ;
35
35
import org .elasticsearch .search .builder .PointInTimeBuilder ;
36
- import org .elasticsearch .search .builder .SearchSourceBuilder ;
37
36
import org .elasticsearch .search .internal .AliasFilter ;
38
37
import org .elasticsearch .search .internal .SearchContext ;
39
38
import org .elasticsearch .search .internal .ShardSearchContextId ;
@@ -88,18 +87,18 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
88
87
private final SetOnce <AtomicArray <ShardSearchFailure >> shardFailures = new SetOnce <>();
89
88
private final Object shardFailuresMutex = new Object ();
90
89
private final AtomicBoolean hasShardResponse = new AtomicBoolean (false );
91
- private final AtomicInteger successfulOps = new AtomicInteger () ;
90
+ private final AtomicInteger successfulOps ;
92
91
private final SearchTimeProvider timeProvider ;
93
92
private final SearchResponse .Clusters clusters ;
94
93
95
- protected final List <SearchShardIterator > toSkipShardsIts ;
96
94
protected final List <SearchShardIterator > shardsIts ;
97
95
private final SearchShardIterator [] shardIterators ;
98
96
private final AtomicInteger outstandingShards ;
99
97
private final int maxConcurrentRequestsPerNode ;
100
98
private final Map <String , PendingExecutions > pendingExecutionsPerNode = new ConcurrentHashMap <>();
101
99
private final boolean throttleConcurrentRequests ;
102
100
private final AtomicBoolean requestCancelled = new AtomicBoolean ();
101
+ private final int skippedCount ;
103
102
104
103
// protected for tests
105
104
protected final List <Releasable > releasables = new ArrayList <>();
@@ -125,18 +124,19 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
125
124
) {
126
125
super (name );
127
126
this .namedWriteableRegistry = namedWriteableRegistry ;
128
- final List <SearchShardIterator > toSkipIterators = new ArrayList <>();
129
127
final List <SearchShardIterator > iterators = new ArrayList <>();
128
+ int skipped = 0 ;
130
129
for (final SearchShardIterator iterator : shardsIts ) {
131
130
if (iterator .skip ()) {
132
- toSkipIterators . add ( iterator ) ;
131
+ skipped ++ ;
133
132
} else {
134
133
iterators .add (iterator );
135
134
}
136
135
}
137
- this .toSkipShardsIts = toSkipIterators ;
136
+ this .skippedCount = skipped ;
138
137
this .shardsIts = iterators ;
139
- outstandingShards = new AtomicInteger (shardsIts .size ());
138
+ outstandingShards = new AtomicInteger (shardsIts .size () - skipped );
139
+ successfulOps = new AtomicInteger (skipped );
140
140
this .shardIterators = iterators .toArray (new SearchShardIterator [0 ]);
141
141
// we later compute the shard index based on the natural order of the shards
142
142
// that participate in the search request. This means that this number is
@@ -167,11 +167,19 @@ abstract class AbstractSearchAsyncAction<Result extends SearchPhaseResult> exten
167
167
protected void notifyListShards (
168
168
SearchProgressListener progressListener ,
169
169
SearchResponse .Clusters clusters ,
170
- SearchSourceBuilder sourceBuilder
170
+ SearchRequest searchRequest ,
171
+ List <SearchShardIterator > allIterators
171
172
) {
173
+ final List <SearchShardIterator > skipped = new ArrayList <>();
174
+ for (SearchShardIterator iter : allIterators ) {
175
+ if (iter .skip ()) {
176
+ skipped .add (iter );
177
+ }
178
+ }
179
+ var sourceBuilder = searchRequest .source ();
172
180
progressListener .notifyListShards (
173
181
SearchProgressListener .buildSearchShardsFromIter (this .shardsIts ),
174
- SearchProgressListener .buildSearchShardsFromIter (toSkipShardsIts ),
182
+ SearchProgressListener .buildSearchShardsFromIter (skipped ),
175
183
clusters ,
176
184
sourceBuilder == null || sourceBuilder .size () > 0 ,
177
185
timeProvider
@@ -215,10 +223,6 @@ public final void start() {
215
223
216
224
@ Override
217
225
protected final void run () {
218
- for (final SearchShardIterator iterator : toSkipShardsIts ) {
219
- assert iterator .skip ();
220
- skipShard (iterator );
221
- }
222
226
final Map <SearchShardIterator , Integer > shardIndexMap = Maps .newHashMapWithExpectedSize (shardIterators .length );
223
227
for (int i = 0 ; i < shardIterators .length ; i ++) {
224
228
shardIndexMap .put (shardIterators [i ], i );
@@ -237,15 +241,11 @@ protected final void run() {
237
241
performPhaseOnShard (shardIndex , shardRoutings , routing );
238
242
}
239
243
}
244
+ } else {
245
+ onPhaseDone ();
240
246
}
241
247
}
242
248
243
- void skipShard (SearchShardIterator iterator ) {
244
- successfulOps .incrementAndGet ();
245
- assert iterator .skip ();
246
- successfulShardExecution ();
247
- }
248
-
249
249
private void performPhaseOnShard (final int shardIndex , final SearchShardIterator shardIt , final SearchShardTarget shard ) {
250
250
if (throttleConcurrentRequests ) {
251
251
var pendingExecutions = pendingExecutionsPerNode .computeIfAbsent (
@@ -343,7 +343,7 @@ protected void executeNextPhase(String currentPhase, Supplier<SearchPhase> nextP
343
343
"Partial shards failure (unavailable: {}, successful: {}, skipped: {}, num-shards: {}, phase: {})" ,
344
344
discrepancy ,
345
345
successfulOps .get (),
346
- toSkipShardsIts . size () ,
346
+ skippedCount ,
347
347
getNumShards (),
348
348
currentPhase
349
349
);
@@ -585,7 +585,7 @@ private SearchResponse buildSearchResponse(
585
585
scrollId ,
586
586
getNumShards (),
587
587
numSuccess ,
588
- toSkipShardsIts . size () ,
588
+ skippedCount ,
589
589
buildTookInMillis (),
590
590
failures ,
591
591
clusters ,
0 commit comments