-
Notifications
You must be signed in to change notification settings - Fork 24.2k
/
TransportShardReplicationOperationAction.java
857 lines (751 loc) · 36.9 KB
/
TransportShardReplicationOperationAction.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.support.replication;
import org.elasticsearch.*;
import org.elasticsearch.action.*;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.TransportAction;
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.ClusterStateObserver;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.index.engine.DocumentAlreadyExistsException;
import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.service.IndexService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.service.IndexShard;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
import java.io.IOException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
/**
*/
public abstract class TransportShardReplicationOperationAction<Request extends ShardReplicationOperationRequest, ReplicaRequest extends ShardReplicationOperationRequest, Response extends ActionResponse> extends TransportAction<Request, Response> {
protected final TransportService transportService;
protected final ClusterService clusterService;
protected final IndicesService indicesService;
protected final ShardStateAction shardStateAction;
protected final ReplicationType defaultReplicationType;
protected final WriteConsistencyLevel defaultWriteConsistencyLevel;
protected final TransportRequestOptions transportOptions;
final String transportReplicaAction;
final String executor;
final boolean checkWriteConsistency;
protected TransportShardReplicationOperationAction(Settings settings, String actionName, TransportService transportService,
ClusterService clusterService, IndicesService indicesService,
ThreadPool threadPool, ShardStateAction shardStateAction, ActionFilters actionFilters) {
super(settings, actionName, threadPool, actionFilters);
this.transportService = transportService;
this.clusterService = clusterService;
this.indicesService = indicesService;
this.shardStateAction = shardStateAction;
this.transportReplicaAction = actionName + "[r]";
this.executor = executor();
this.checkWriteConsistency = checkWriteConsistency();
transportService.registerHandler(actionName, new OperationTransportHandler());
transportService.registerHandler(transportReplicaAction, new ReplicaOperationTransportHandler());
this.transportOptions = transportOptions();
this.defaultReplicationType = ReplicationType.fromString(settings.get("action.replication_type", "sync"));
this.defaultWriteConsistencyLevel = WriteConsistencyLevel.fromString(settings.get("action.write_consistency", "quorum"));
}
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
new AsyncShardOperationAction(request, listener).start();
}
protected abstract Request newRequestInstance();
protected abstract ReplicaRequest newReplicaRequestInstance();
protected abstract Response newResponseInstance();
protected abstract String executor();
protected abstract PrimaryResponse<Response, ReplicaRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest);
protected abstract void shardOperationOnReplica(ReplicaOperationRequest shardRequest);
/**
* Called once replica operations have been dispatched on the
*/
protected void postPrimaryOperation(InternalRequest request, PrimaryResponse<Response, ReplicaRequest> response) {
}
protected abstract ShardIterator shards(ClusterState clusterState, InternalRequest request) throws ElasticsearchException;
protected abstract boolean checkWriteConsistency();
protected ClusterBlockException checkGlobalBlock(ClusterState state) {
return state.blocks().globalBlockedException(ClusterBlockLevel.WRITE);
}
protected ClusterBlockException checkRequestBlock(ClusterState state, InternalRequest request) {
return state.blocks().indexBlockedException(ClusterBlockLevel.WRITE, request.concreteIndex());
}
protected abstract boolean resolveIndex();
/**
* Resolves the request, by default doing nothing. If the resolve
* means a different execution, then return false here to indicate not to continue and execute this request.
*/
protected boolean resolveRequest(ClusterState state, InternalRequest request, ActionListener<Response> listener) {
return true;
}
protected TransportRequestOptions transportOptions() {
return TransportRequestOptions.EMPTY;
}
/**
* Should the operations be performed on the replicas as well. Defaults to <tt>false</tt> meaning operations
* will be executed on the replica.
*/
protected boolean ignoreReplicas() {
return false;
}
protected boolean retryPrimaryException(Throwable e) {
return TransportActions.isShardNotAvailableException(e);
}
/**
* Should an exception be ignored when the operation is performed on the replica.
*/
protected boolean ignoreReplicaException(Throwable e) {
if (TransportActions.isShardNotAvailableException(e)) {
return true;
}
// on version conflict or document missing, it means
// that a new change has crept into the replica, and it's fine
if (isConflictException(e)) {
return true;
}
return false;
}
protected boolean isConflictException(Throwable e) {
Throwable cause = ExceptionsHelper.unwrapCause(e);
// on version conflict or document missing, it means
// that a new change has crept into the replica, and it's fine
if (cause instanceof VersionConflictEngineException) {
return true;
}
if (cause instanceof DocumentAlreadyExistsException) {
return true;
}
return false;
}
class OperationTransportHandler extends BaseTransportRequestHandler<Request> {
@Override
public Request newInstance() {
return newRequestInstance();
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
@Override
public void messageReceived(final Request request, final TransportChannel channel) throws Exception {
// no need to have a threaded listener since we just send back a response
request.listenerThreaded(false);
// if we have a local operation, execute it on a thread since we don't spawn
request.operationThreaded(true);
execute(request, new ActionListener<Response>() {
@Override
public void onResponse(Response result) {
try {
channel.sendResponse(result);
} catch (Throwable e) {
onFailure(e);
}
}
@Override
public void onFailure(Throwable e) {
try {
channel.sendResponse(e);
} catch (Throwable e1) {
logger.warn("Failed to send response for " + actionName, e1);
}
}
});
}
}
class ReplicaOperationTransportHandler extends BaseTransportRequestHandler<ReplicaOperationRequest> {
@Override
public ReplicaOperationRequest newInstance() {
return new ReplicaOperationRequest();
}
@Override
public String executor() {
return executor;
}
// we must never reject on because of thread pool capacity on replicas
@Override
public boolean isForceExecution() {
return true;
}
@Override
public void messageReceived(final ReplicaOperationRequest request, final TransportChannel channel) throws Exception {
try {
shardOperationOnReplica(request);
} catch (Throwable t) {
failReplicaIfNeeded(request.shardId.getIndex(), request.shardId.id(), t);
throw t;
}
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}
}
protected class PrimaryOperationRequest {
public ShardId shardId;
public Request request;
public PrimaryOperationRequest(int shardId, String index, Request request) {
this.shardId = new ShardId(index, shardId);
this.request = request;
}
}
protected class ReplicaOperationRequest extends TransportRequest implements IndicesRequest {
public ShardId shardId;
public ReplicaRequest request;
ReplicaOperationRequest() {
}
ReplicaOperationRequest(ShardId shardId, ReplicaRequest request) {
super(request);
this.shardId = shardId;
this.request = request;
}
@Override
public String[] indices() {
return request.indices();
}
@Override
public IndicesOptions indicesOptions() {
return request.indicesOptions();
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
int shard = -1;
if (in.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
shardId = ShardId.readShardId(in);
} else {
shard = in.readVInt();
}
request = newReplicaRequestInstance();
request.readFrom(in);
if (in.getVersion().before(Version.V_1_4_0_Beta1)) {
assert shard >= 0;
//older nodes will send the concrete index as part of the request
shardId = new ShardId(request.index(), shard);
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
if (out.getVersion().onOrAfter(Version.V_1_4_0_Beta1)) {
shardId.writeTo(out);
} else {
out.writeVInt(shardId.id());
//older nodes expect the concrete index as part of the request
request.index(shardId.getIndex());
}
request.writeTo(out);
}
}
protected class AsyncShardOperationAction {
private final ActionListener<Response> listener;
private final InternalRequest internalRequest;
private volatile ShardIterator shardIt;
private final AtomicBoolean primaryOperationStarted = new AtomicBoolean();
private final ReplicationType replicationType;
private volatile ClusterStateObserver observer;
AsyncShardOperationAction(Request request, ActionListener<Response> listener) {
this.internalRequest = new InternalRequest(request);
this.listener = listener;
if (request.replicationType() != ReplicationType.DEFAULT) {
replicationType = request.replicationType();
} else {
replicationType = defaultReplicationType;
}
}
public void start() {
this.observer = new ClusterStateObserver(clusterService, internalRequest.request().timeout(), logger);
doStart();
}
/**
* Returns <tt>true</tt> if the action starting to be performed on the primary (or is done).
*/
protected void doStart() throws ElasticsearchException {
try {
ClusterBlockException blockException = checkGlobalBlock(observer.observedState());
if (blockException != null) {
if (blockException.retryable()) {
logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
retry(blockException);
return;
} else {
throw blockException;
}
}
if (resolveIndex()) {
internalRequest.concreteIndex(observer.observedState().metaData().concreteSingleIndex(internalRequest.request().index(), internalRequest.request().indicesOptions()));
} else {
internalRequest.concreteIndex(internalRequest.request().index());
}
// check if we need to execute, and if not, return
if (!resolveRequest(observer.observedState(), internalRequest, listener)) {
return;
}
blockException = checkRequestBlock(observer.observedState(), internalRequest);
if (blockException != null) {
if (blockException.retryable()) {
logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
retry(blockException);
return;
} else {
throw blockException;
}
}
shardIt = shards(observer.observedState(), internalRequest);
} catch (Throwable e) {
listener.onFailure(e);
return;
}
// no shardIt, might be in the case between index gateway recovery and shardIt initialization
if (shardIt.size() == 0) {
logger.trace("no shard instances known for shard [{}], scheduling a retry", shardIt.shardId());
retryBecauseUnavailable(shardIt.shardId(), "No active shards.");
return;
}
boolean foundPrimary = false;
ShardRouting shardX;
while ((shardX = shardIt.nextOrNull()) != null) {
final ShardRouting shard = shardX;
// we only deal with primary shardIt here...
if (!shard.primary()) {
continue;
}
if (!shard.active() || !observer.observedState().nodes().nodeExists(shard.currentNodeId())) {
logger.trace("primary shard [{}] is not yet active or we do not know the node it is assigned to [{}], scheduling a retry.", shard.shardId(), shard.currentNodeId());
retryBecauseUnavailable(shardIt.shardId(), "Primary shard is not active or isn't assigned is a known node.");
return;
}
if (!primaryOperationStarted.compareAndSet(false, true)) {
return;
}
foundPrimary = true;
if (shard.currentNodeId().equals(observer.observedState().nodes().localNodeId())) {
try {
if (internalRequest.request().operationThreaded()) {
internalRequest.request().beforeLocalFork();
threadPool.executor(executor).execute(new Runnable() {
@Override
public void run() {
try {
performOnPrimary(shard.id(), shard);
} catch (Throwable t) {
listener.onFailure(t);
}
}
});
} else {
performOnPrimary(shard.id(), shard);
}
} catch (Throwable t) {
listener.onFailure(t);
}
} else {
DiscoveryNode node = observer.observedState().nodes().get(shard.currentNodeId());
transportService.sendRequest(node, actionName, internalRequest.request(), transportOptions, new BaseTransportResponseHandler<Response>() {
@Override
public Response newInstance() {
return newResponseInstance();
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
@Override
public void handleResponse(Response response) {
listener.onResponse(response);
}
@Override
public void handleException(TransportException exp) {
// if we got disconnected from the node, or the node / shard is not in the right state (being closed)
if (exp.unwrapCause() instanceof ConnectTransportException || exp.unwrapCause() instanceof NodeClosedException ||
retryPrimaryException(exp)) {
primaryOperationStarted.set(false);
// we already marked it as started when we executed it (removed the listener) so pass false
// to re-add to the cluster listener
logger.trace("received an error from node the primary was assigned to ({}), scheduling a retry", exp.getMessage());
retry(exp);
} else {
listener.onFailure(exp);
}
}
});
}
break;
}
// we won't find a primary if there are no shards in the shard iterator, retry...
if (!foundPrimary) {
logger.trace("couldn't find a eligible primary shard, scheduling for retry.");
retryBecauseUnavailable(shardIt.shardId(), "No active shards.");
}
}
void retry(Throwable failure) {
assert failure != null;
if (observer.isTimedOut()) {
// we running as a last attempt after a timeout has happened. don't retry
listener.onFailure(failure);
return;
}
// make it threaded operation so we fork on the discovery listener thread
internalRequest.request().beforeLocalFork();
internalRequest.request().operationThreaded(true);
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
doStart();
}
@Override
public void onClusterServiceClose() {
listener.onFailure(new NodeClosedException(clusterService.localNode()));
}
@Override
public void onTimeout(TimeValue timeout) {
// Try one more time...
doStart();
}
});
}
void performOnPrimary(int primaryShardId, final ShardRouting shard) {
ClusterState clusterState = observer.observedState();
if (raiseFailureIfHaveNotEnoughActiveShardCopies(shard, clusterState)) {
return;
}
try {
PrimaryResponse<Response, ReplicaRequest> response = shardOperationOnPrimary(clusterState, new PrimaryOperationRequest(primaryShardId, internalRequest.concreteIndex(), internalRequest.request()));
performReplicas(response);
} catch (Throwable e) {
internalRequest.request.setCanHaveDuplicates();
// shard has not been allocated yet, retry it here
if (retryPrimaryException(e)) {
primaryOperationStarted.set(false);
logger.trace("had an error while performing operation on primary ({}), scheduling a retry.", e.getMessage());
retry(e);
return;
}
if (e instanceof ElasticsearchException && ((ElasticsearchException) e).status() == RestStatus.CONFLICT) {
if (logger.isTraceEnabled()) {
logger.trace(shard.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
}
} else {
if (logger.isDebugEnabled()) {
logger.debug(shard.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
}
}
listener.onFailure(e);
}
}
void performReplicas(final PrimaryResponse<Response, ReplicaRequest> response) {
if (ignoreReplicas()) {
postPrimaryOperation(internalRequest, response);
listener.onResponse(response.response());
return;
}
ShardRouting shard;
// we double check on the state, if it got changed we need to make sure we take the latest one cause
// maybe a replica shard started its recovery process and we need to apply it there...
// we also need to make sure if the new state has a new primary shard (that we indexed to before) started
// and assigned to another node (while the indexing happened). In that case, we want to apply it on the
// new primary shard as well...
ClusterState newState = clusterService.state();
ShardRouting newPrimaryShard = null;
if (observer.observedState() != newState) {
shardIt.reset();
ShardRouting originalPrimaryShard = null;
while ((shard = shardIt.nextOrNull()) != null) {
if (shard.primary()) {
originalPrimaryShard = shard;
break;
}
}
if (originalPrimaryShard == null || !originalPrimaryShard.active()) {
throw new ElasticsearchIllegalStateException("unexpected state, failed to find primary shard on an index operation that succeeded");
}
observer.reset(newState);
shardIt = shards(newState, internalRequest);
while ((shard = shardIt.nextOrNull()) != null) {
if (shard.primary()) {
if (originalPrimaryShard.currentNodeId().equals(shard.currentNodeId())) {
newPrimaryShard = null;
} else {
newPrimaryShard = shard;
}
break;
}
}
shardIt.reset();
internalRequest.request().setCanHaveDuplicates(); // safe side, cluster state changed, we might have dups
} else {
shardIt.reset();
while ((shard = shardIt.nextOrNull()) != null) {
if (shard.state() != ShardRoutingState.STARTED) {
internalRequest.request().setCanHaveDuplicates();
}
}
shardIt.reset();
}
// initialize the counter
int replicaCounter = shardIt.assignedReplicasIncludingRelocating();
if (newPrimaryShard != null) {
replicaCounter++;
}
if (replicaCounter == 0) {
postPrimaryOperation(internalRequest, response);
listener.onResponse(response.response());
return;
}
if (replicationType == ReplicationType.ASYNC) {
postPrimaryOperation(internalRequest, response);
// async replication, notify the listener
listener.onResponse(response.response());
// now, trick the counter so it won't decrease to 0 and notify the listeners
replicaCounter = Integer.MIN_VALUE;
}
// we add one to the replica count to do the postPrimaryOperation
replicaCounter++;
AtomicInteger counter = new AtomicInteger(replicaCounter);
IndexMetaData indexMetaData = observer.observedState().metaData().index(internalRequest.concreteIndex());
if (newPrimaryShard != null) {
performOnReplica(response, counter, newPrimaryShard, newPrimaryShard.currentNodeId(), indexMetaData);
}
shardIt.reset(); // reset the iterator
while ((shard = shardIt.nextOrNull()) != null) {
// if its unassigned, nothing to do here...
if (shard.unassigned()) {
continue;
}
// if the shard is primary and relocating, add one to the counter since we perform it on the replica as well
// (and we already did it on the primary)
boolean doOnlyOnRelocating = false;
if (shard.primary()) {
if (shard.relocating()) {
doOnlyOnRelocating = true;
} else {
continue;
}
}
// we index on a replica that is initializing as well since we might not have got the event
// yet that it was started. We will get an exception IllegalShardState exception if its not started
// and that's fine, we will ignore it
if (!doOnlyOnRelocating) {
performOnReplica(response, counter, shard, shard.currentNodeId(), indexMetaData);
}
if (shard.relocating()) {
performOnReplica(response, counter, shard, shard.relocatingNodeId(), indexMetaData);
}
}
// now do the postPrimary operation, and check if the listener needs to be invoked
postPrimaryOperation(internalRequest, response);
// we also invoke here in case replicas finish before postPrimaryAction does
if (counter.decrementAndGet() == 0) {
listener.onResponse(response.response());
}
}
void performOnReplica(final PrimaryResponse<Response, ReplicaRequest> response, final AtomicInteger counter, final ShardRouting shard, String nodeId, final IndexMetaData indexMetaData) {
// if we don't have that node, it means that it might have failed and will be created again, in
// this case, we don't have to do the operation, and just let it failover
if (!observer.observedState().nodes().nodeExists(nodeId)) {
if (counter.decrementAndGet() == 0) {
listener.onResponse(response.response());
}
return;
}
final ReplicaOperationRequest shardRequest = new ReplicaOperationRequest(shardIt.shardId(), response.replicaRequest());
if (!nodeId.equals(observer.observedState().nodes().localNodeId())) {
final DiscoveryNode node = observer.observedState().nodes().get(nodeId);
transportService.sendRequest(node, transportReplicaAction, shardRequest, transportOptions, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@Override
public void handleResponse(TransportResponse.Empty vResponse) {
finishIfPossible();
}
@Override
public void handleException(TransportException exp) {
logger.trace("[{}] Transport failure during replica request [{}] ", exp, node, internalRequest.request());
if (!ignoreReplicaException(exp)) {
logger.warn("Failed to perform " + actionName + " on remote replica " + node + shardIt.shardId(), exp);
shardStateAction.shardFailed(shard, indexMetaData.getUUID(),
"Failed to perform [" + actionName + "] on replica, message [" + ExceptionsHelper.detailedMessage(exp) + "]");
}
finishIfPossible();
}
private void finishIfPossible() {
if (counter.decrementAndGet() == 0) {
listener.onResponse(response.response());
}
}
});
} else {
if (internalRequest.request().operationThreaded()) {
internalRequest.request().beforeLocalFork();
try {
threadPool.executor(executor).execute(new AbstractRunnable() {
@Override
protected void doRun() {
try {
shardOperationOnReplica(shardRequest);
} catch (Throwable e) {
failReplicaIfNeeded(shard.index(), shard.id(), e);
}
if (counter.decrementAndGet() == 0) {
listener.onResponse(response.response());
}
}
// we must never reject on because of thread pool capacity on replicas
@Override
public boolean isForceExecution() {
return true;
}
@Override
public void onFailure(Throwable t) {
}
});
} catch (Throwable e) {
failReplicaIfNeeded(shard.index(), shard.id(), e);
// we want to decrement the counter here, in teh failure handling, cause we got rejected
// from executing on the thread pool
if (counter.decrementAndGet() == 0) {
listener.onResponse(response.response());
}
}
} else {
try {
shardOperationOnReplica(shardRequest);
} catch (Throwable e) {
failReplicaIfNeeded(shard.index(), shard.id(), e);
}
if (counter.decrementAndGet() == 0) {
listener.onResponse(response.response());
}
}
}
}
boolean raiseFailureIfHaveNotEnoughActiveShardCopies(ShardRouting shard, ClusterState state) {
if (!checkWriteConsistency) {
return false;
}
final WriteConsistencyLevel consistencyLevel;
if (internalRequest.request().consistencyLevel() != WriteConsistencyLevel.DEFAULT) {
consistencyLevel = internalRequest.request().consistencyLevel();
} else {
consistencyLevel = defaultWriteConsistencyLevel;
}
final int sizeActive;
final int requiredNumber;
IndexRoutingTable indexRoutingTable = state.getRoutingTable().index(shard.index());
if (indexRoutingTable != null) {
IndexShardRoutingTable shardRoutingTable = indexRoutingTable.shard(shard.getId());
if (shardRoutingTable != null) {
sizeActive = shardRoutingTable.activeShards().size();
if (consistencyLevel == WriteConsistencyLevel.QUORUM && shardRoutingTable.getSize() > 2) {
// only for more than 2 in the number of shardIt it makes sense, otherwise its 1 shard with 1 replica, quorum is 1 (which is what it is initialized to)
requiredNumber = (shardRoutingTable.getSize() / 2) + 1;
} else if (consistencyLevel == WriteConsistencyLevel.ALL) {
requiredNumber = shardRoutingTable.getSize();
} else {
requiredNumber = 1;
}
} else {
sizeActive = 0;
requiredNumber = 1;
}
} else {
sizeActive = 0;
requiredNumber = 1;
}
if (sizeActive < requiredNumber) {
logger.trace("not enough active copies of shard [{}] to meet write consistency of [{}] (have {}, needed {}), scheduling a retry.",
shard.shardId(), consistencyLevel, sizeActive, requiredNumber);
primaryOperationStarted.set(false);
// A dedicated exception would be nice...
retryBecauseUnavailable(shard.shardId(), "Not enough active copies to meet write consistency of [" + consistencyLevel + "] (have " + sizeActive + ", needed " + requiredNumber + ").");
return true;
} else {
return false;
}
}
void retryBecauseUnavailable(ShardId shardId, String message) {
retry(new UnavailableShardsException(shardId, message + " Timeout: [" + internalRequest.request().timeout() +"], request: " + internalRequest.request().toString()));
}
}
private void failReplicaIfNeeded(String index, int shardId, Throwable t) {
logger.trace("failure on replica [{}][{}]", t, index, shardId);
if (!ignoreReplicaException(t)) {
IndexService indexService = indicesService.indexService(index);
if (indexService == null) {
logger.debug("ignoring failed replica [{}][{}] because index was already removed.", index, shardId);
return;
}
IndexShard indexShard = indexService.shard(shardId);
if (indexShard == null) {
logger.debug("ignoring failed replica [{}][{}] because index was already removed.", index, shardId);
return;
}
indexShard.failShard(actionName + " failed on replica", t);
}
}
public static class PrimaryResponse<Response, ReplicaRequest> {
private final ReplicaRequest replicaRequest;
private final Response response;
private final Object payload;
public PrimaryResponse(ReplicaRequest replicaRequest, Response response, Object payload) {
this.replicaRequest = replicaRequest;
this.response = response;
this.payload = payload;
}
public ReplicaRequest replicaRequest() {
return this.replicaRequest;
}
public Response response() {
return response;
}
public Object payload() {
return payload;
}
}
/**
* Internal request class that gets built on each node. Holds the original request plus additional info.
*/
protected class InternalRequest {
final Request request;
String concreteIndex;
InternalRequest(Request request) {
this.request = request;
}
public Request request() {
return request;
}
void concreteIndex(String concreteIndex) {
this.concreteIndex = concreteIndex;
}
public String concreteIndex() {
return concreteIndex;
}
}
}