forked from elastic/elasticsearch
/
TransportReplicationAction.java
1109 lines (984 loc) · 50.3 KB
/
TransportReplicationAction.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
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* 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.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionWriteResponse;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexRequest.OpType;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.ActionFilters;
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.index.MappingUpdatedAction;
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.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.compress.CompressedString;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.DocumentAlreadyExistsException;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardException;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.river.RiverIndexName;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*;
import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
/**
*/
public abstract class TransportReplicationAction<Request extends ReplicationRequest, ReplicaRequest extends ReplicationRequest, Response extends ActionWriteResponse> extends TransportAction<Request, Response> {
protected final TransportService transportService;
protected final ClusterService clusterService;
protected final IndicesService indicesService;
protected final ShardStateAction shardStateAction;
protected final WriteConsistencyLevel defaultWriteConsistencyLevel;
protected final TransportRequestOptions transportOptions;
protected final MappingUpdatedAction mappingUpdatedAction;
final String transportReplicaAction;
final String executor;
final boolean checkWriteConsistency;
protected TransportReplicationAction(Settings settings, String actionName, TransportService transportService,
ClusterService clusterService, IndicesService indicesService,
ThreadPool threadPool, ShardStateAction shardStateAction,
MappingUpdatedAction mappingUpdatedAction, ActionFilters actionFilters,
Class<Request> request, Class<ReplicaRequest> replicaRequest, String executor) {
super(settings, actionName, threadPool, actionFilters);
this.transportService = transportService;
this.clusterService = clusterService;
this.indicesService = indicesService;
this.shardStateAction = shardStateAction;
this.mappingUpdatedAction = mappingUpdatedAction;
this.transportReplicaAction = actionName + "[r]";
this.executor = executor;
this.checkWriteConsistency = checkWriteConsistency();
transportService.registerRequestHandler(actionName, request, ThreadPool.Names.SAME, new OperationTransportHandler());
// we must never reject on because of thread pool capacity on replicas
transportService.registerRequestHandler(transportReplicaAction, replicaRequest, executor, true, new ReplicaOperationTransportHandler());
this.transportOptions = transportOptions();
this.defaultWriteConsistencyLevel = WriteConsistencyLevel.fromString(settings.get("action.write_consistency", "quorum"));
}
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
new PrimaryPhase(request, listener).run();
}
protected abstract Response newResponseInstance();
/**
* @return A tuple containing not null values, as first value the result of the primary operation and as second value
* the request to be executed on the replica shards.
*/
protected abstract Tuple<Response, ReplicaRequest> shardOperationOnPrimary(ClusterState clusterState, PrimaryOperationRequest shardRequest) throws Throwable;
protected abstract void shardOperationOnReplica(ShardId shardId, ReplicaRequest shardRequest);
protected abstract ShardIterator shards(ClusterState clusterState, InternalRequest request);
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. Can be subclassed to do
* additional processing or validation depending on the incoming request
*/
protected void resolveRequest(ClusterState state, InternalRequest request, ActionListener<Response> listener) {
}
protected TransportRequestOptions transportOptions() {
return TransportRequestOptions.EMPTY;
}
protected boolean retryPrimaryException(Throwable e) {
return e.getClass() == RetryOnPrimaryException.class
|| 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;
}
protected static class WriteResult<T extends ActionWriteResponse> {
public final T response;
public final Translog.Location location;
public WriteResult(T response, Translog.Location location) {
this.response = response;
this.location = location;
}
@SuppressWarnings("unchecked")
public <T extends ActionWriteResponse> T response() {
// this sets total, pending and failed to 0 and this is ok, because we will embed this into the replica
// request and not use it
response.setShardInfo(new ActionWriteResponse.ShardInfo());
return (T) response;
}
}
class OperationTransportHandler implements TransportRequestHandler<Request> {
@Override
public void messageReceived(final Request request, final TransportChannel channel) throws Exception {
// 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 implements TransportRequestHandler<ReplicaRequest> {
@Override
public void messageReceived(final ReplicaRequest request, final TransportChannel channel) throws Exception {
new AsyncReplicaAction(request, channel).run();
}
}
protected static class RetryOnReplicaException extends IndexShardException {
public RetryOnReplicaException(ShardId shardId, String msg) {
super(shardId, msg);
}
public RetryOnReplicaException(ShardId shardId, String msg, Throwable cause) {
super(shardId, msg, cause);
}
}
private final class AsyncReplicaAction extends AbstractRunnable {
private final ReplicaRequest request;
private final TransportChannel channel;
// important: we pass null as a timeout as failing a replica is
// something we want to avoid at all costs
private final ClusterStateObserver observer = new ClusterStateObserver(clusterService, null, logger);
AsyncReplicaAction(ReplicaRequest request, TransportChannel channel) {
this.request = request;
this.channel = channel;
}
@Override
public void onFailure(Throwable t) {
if (t instanceof RetryOnReplicaException) {
logger.trace("Retrying operation on replica", t);
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
threadPool.executor(executor).execute(AsyncReplicaAction.this);
}
@Override
public void onClusterServiceClose() {
responseWithFailure(new NodeClosedException(clusterService.localNode()));
}
@Override
public void onTimeout(TimeValue timeout) {
throw new AssertionError("Cannot happen: there is not timeout");
}
});
} else {
try {
failReplicaIfNeeded(request.internalShardId.getIndex(), request.internalShardId.id(), t);
} catch (Throwable unexpected) {
logger.error("{} unexpected error while failing replica", request.internalShardId.id(), unexpected);
} finally {
responseWithFailure(t);
}
}
}
protected void responseWithFailure(Throwable t) {
try {
channel.sendResponse(t);
} catch (IOException responseException) {
logger.warn("failed to send error message back to client for action [" + transportReplicaAction + "]", responseException);
logger.warn("actual Exception", t);
}
}
@Override
protected void doRun() throws Exception {
try (Releasable shardReference = getIndexShardOperationsCounter(request.internalShardId)) {
shardOperationOnReplica(request.internalShardId, request);
}
channel.sendResponse(TransportResponse.Empty.INSTANCE);
}
}
protected class PrimaryOperationRequest {
public final ShardId shardId;
public final Request request;
public PrimaryOperationRequest(int shardId, String index, Request request) {
this.shardId = new ShardId(index, shardId);
this.request = request;
}
}
protected static class RetryOnPrimaryException extends IndexShardException {
public RetryOnPrimaryException(ShardId shardId, String msg) {
super(shardId, msg);
}
public RetryOnPrimaryException(ShardId shardId, String msg, Throwable cause) {
super(shardId, msg, cause);
}
}
/**
* Responsible for performing all operations up to the point we start starting sending requests to replica shards.
* Including forwarding the request to another node if the primary is not assigned locally.
* <p/>
* Note that as soon as we start sending request to replicas, state responsibility is transferred to {@link ReplicationPhase}
*/
final class PrimaryPhase extends AbstractRunnable {
private final ActionListener<Response> listener;
private final InternalRequest internalRequest;
private final ClusterStateObserver observer;
private final AtomicBoolean finished = new AtomicBoolean(false);
private volatile Releasable indexShardReference;
PrimaryPhase(Request request, ActionListener<Response> listener) {
this.internalRequest = new InternalRequest(request);
this.listener = listener;
this.observer = new ClusterStateObserver(clusterService, internalRequest.request().timeout(), logger);
}
@Override
public void onFailure(Throwable e) {
finishWithUnexpectedFailure(e);
}
protected void doRun() {
if (checkBlocks() == false) {
return;
}
final ShardIterator shardIt = shards(observer.observedState(), internalRequest);
final ShardRouting primary = resolvePrimary(shardIt);
if (primary == null) {
retryBecauseUnavailable(shardIt.shardId(), "No active shards.");
return;
}
if (primary.active() == false) {
logger.trace("primary shard [{}] is not yet active, scheduling a retry.", primary.shardId());
retryBecauseUnavailable(shardIt.shardId(), "Primary shard is not active or isn't assigned to a known node.");
return;
}
if (observer.observedState().nodes().nodeExists(primary.currentNodeId()) == false) {
logger.trace("primary shard [{}] is assigned to anode we do not know the node, scheduling a retry.", primary.shardId(), primary.currentNodeId());
retryBecauseUnavailable(shardIt.shardId(), "Primary shard is not active or isn't assigned to a known node.");
return;
}
routeRequestOrPerformLocally(primary, shardIt);
}
/**
* checks for any cluster state blocks. Returns true if operation is OK to proceeded.
* if false is return, no further action is needed. The method takes care of any continuation, by either
* responding to the listener or scheduling a retry
*/
protected boolean checkBlocks() {
ClusterBlockException blockException = checkGlobalBlock(observer.observedState());
if (blockException != null) {
if (blockException.retryable()) {
logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
retry(blockException);
} else {
finishAsFailed(blockException);
}
return false;
}
if (resolveIndex()) {
internalRequest.concreteIndex(observer.observedState().metaData().concreteSingleIndex(internalRequest.request().index(), internalRequest.request().indicesOptions()));
} else {
internalRequest.concreteIndex(internalRequest.request().index());
}
resolveRequest(observer.observedState(), internalRequest, listener);
blockException = checkRequestBlock(observer.observedState(), internalRequest);
if (blockException != null) {
if (blockException.retryable()) {
logger.trace("cluster is blocked ({}), scheduling a retry", blockException.getMessage());
retry(blockException);
} else {
finishAsFailed(blockException);
}
return false;
}
return true;
}
protected ShardRouting resolvePrimary(ShardIterator shardIt) {
// no shardIt, might be in the case between index gateway recovery and shardIt initialization
ShardRouting shard;
while ((shard = shardIt.nextOrNull()) != null) {
// we only deal with primary shardIt here...
if (shard.primary()) {
return shard;
}
}
return null;
}
/**
* send the request to the node holding the primary or execute if local
*/
protected void routeRequestOrPerformLocally(final ShardRouting primary, final ShardIterator shardsIt) {
if (primary.currentNodeId().equals(observer.observedState().nodes().localNodeId())) {
try {
if (internalRequest.request().operationThreaded()) {
threadPool.executor(executor).execute(new AbstractRunnable() {
@Override
public void onFailure(Throwable t) {
finishAsFailed(t);
}
@Override
protected void doRun() throws Exception {
performOnPrimary(primary, shardsIt);
}
});
} else {
performOnPrimary(primary, shardsIt);
}
} catch (Throwable t) {
// no commit: check threadpool rejection.
finishAsFailed(t);
}
} else {
DiscoveryNode node = observer.observedState().nodes().get(primary.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) {
finishOnRemoteSuccess(response);
}
@Override
public void handleException(TransportException exp) {
try {
// 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)) {
internalRequest.request().setCanHaveDuplicates();
// 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 {
finishAsFailed(exp);
}
} catch (Throwable t) {
finishWithUnexpectedFailure(t);
}
}
});
}
}
void retry(Throwable failure) {
assert failure != null;
if (observer.isTimedOut()) {
// we running as a last attempt after a timeout has happened. don't retry
finishAsFailed(failure);
return;
}
// make it threaded operation so we fork on the discovery listener thread
internalRequest.request().operationThreaded(true);
observer.waitForNextChange(new ClusterStateObserver.Listener() {
@Override
public void onNewClusterState(ClusterState state) {
run();
}
@Override
public void onClusterServiceClose() {
finishAsFailed(new NodeClosedException(clusterService.localNode()));
}
@Override
public void onTimeout(TimeValue timeout) {
// Try one more time...
run();
}
});
}
/**
* upon success, finish the first phase and transfer responsibility to the {@link ReplicationPhase}
*/
void finishAndMoveToReplication(ReplicationPhase replicationPhase) {
if (finished.compareAndSet(false, true)) {
replicationPhase.run();
} else {
assert false : "finishAndMoveToReplication called but operation is already finished";
}
}
void finishAsFailed(Throwable failure) {
if (finished.compareAndSet(false, true)) {
Releasables.close(indexShardReference);
logger.trace("operation failed", failure);
listener.onFailure(failure);
} else {
assert false : "finishAsFailed called but operation is already finished";
}
}
void finishWithUnexpectedFailure(Throwable failure) {
logger.warn("unexpected error during the primary phase for action [{}]", failure, actionName);
if (finished.compareAndSet(false, true)) {
Releasables.close(indexShardReference);
listener.onFailure(failure);
} else {
assert false : "finishWithUnexpectedFailure called but operation is already finished";
}
}
void finishOnRemoteSuccess(Response response) {
if (finished.compareAndSet(false, true)) {
logger.trace("operation succeeded");
listener.onResponse(response);
} else {
assert false : "finishOnRemoteSuccess called but operation is already finished";
}
}
/**
* perform the operation on the node holding the primary
*/
void performOnPrimary(final ShardRouting primary, final ShardIterator shardsIt) {
final String writeConsistencyFailure = checkWriteConsistency(primary);
if (writeConsistencyFailure != null) {
retryBecauseUnavailable(primary.shardId(), writeConsistencyFailure);
return;
}
final ReplicationPhase replicationPhase;
try {
indexShardReference = getIndexShardOperationsCounter(primary.shardId());
PrimaryOperationRequest por = new PrimaryOperationRequest(primary.id(), internalRequest.concreteIndex(), internalRequest.request());
Tuple<Response, ReplicaRequest> primaryResponse = shardOperationOnPrimary(observer.observedState(), por);
logger.trace("operation completed on primary [{}]", primary);
replicationPhase = new ReplicationPhase(shardsIt, primaryResponse.v2(), primaryResponse.v1(), observer, primary, internalRequest, listener, indexShardReference);
} catch (Throwable e) {
internalRequest.request.setCanHaveDuplicates();
// shard has not been allocated yet, retry it here
if (retryPrimaryException(e)) {
logger.trace("had an error while performing operation on primary ({}), scheduling a retry.", e.getMessage());
// We have to close here because when we retry we will increment get a new reference on index shard again and we do not want to
// increment twice.
Releasables.close(indexShardReference);
// We have to reset to null here because whe we retry it might be that we never get to the point where we assign a new reference
// (for example, in case the operation was rejected because queue is full). In this case we would release again once one of the finish methods is called.
indexShardReference = null;
retry(e);
return;
}
if (ExceptionsHelper.status(e) == RestStatus.CONFLICT) {
if (logger.isTraceEnabled()) {
logger.trace(primary.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
}
} else {
if (logger.isDebugEnabled()) {
logger.debug(primary.shortSummary() + ": Failed to execute [" + internalRequest.request() + "]", e);
}
}
finishAsFailed(e);
return;
}
finishAndMoveToReplication(replicationPhase);
}
/**
* checks whether we can perform a write based on the write consistency setting
* returns **null* if OK to proceed, or a string describing the reason to stop
*/
String checkWriteConsistency(ShardRouting shard) {
if (checkWriteConsistency == false) {
return null;
}
final WriteConsistencyLevel consistencyLevel;
if (internalRequest.request().consistencyLevel() != WriteConsistencyLevel.DEFAULT) {
consistencyLevel = internalRequest.request().consistencyLevel();
} else {
consistencyLevel = defaultWriteConsistencyLevel;
}
final int sizeActive;
final int requiredNumber;
IndexRoutingTable indexRoutingTable = observer.observedState().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);
return "Not enough active copies to meet write consistency of [" + consistencyLevel + "] (have " + sizeActive + ", needed " + requiredNumber + ").";
} else {
return null;
}
}
void retryBecauseUnavailable(ShardId shardId, String message) {
retry(new UnavailableShardsException(shardId, message + " Timeout: [" + internalRequest.request().timeout() + "], request: " + internalRequest.request().toString()));
}
}
protected Releasable getIndexShardOperationsCounter(ShardId shardId) {
IndexService indexService = indicesService.indexServiceSafe(shardId.index().getName());
IndexShard indexShard = indexService.shardSafe(shardId.id());
return new IndexShardReference(indexShard);
}
private void failReplicaIfNeeded(String index, int shardId, Throwable t) {
logger.trace("failure on replica [{}][{}]", t, index, shardId);
if (ignoreReplicaException(t) == false) {
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);
}
}
/**
* inner class is responsible for send the requests to all replica shards and manage the responses
*/
final class ReplicationPhase extends AbstractRunnable {
private final ReplicaRequest replicaRequest;
private final Response finalResponse;
private final ShardIterator shardIt;
private final ActionListener<Response> listener;
private final AtomicBoolean finished = new AtomicBoolean(false);
private final AtomicInteger success = new AtomicInteger(1); // We already wrote into the primary shard
private final ConcurrentMap<String, Throwable> shardReplicaFailures = ConcurrentCollections.newConcurrentMap();
private final IndexMetaData indexMetaData;
private final ShardRouting originalPrimaryShard;
private final AtomicInteger pending;
private final int totalShards;
private final ClusterStateObserver observer;
private final Releasable indexShardReference;
/**
* the constructor doesn't take any action, just calculates state. Call {@link #run()} to start
* replicating.
*/
public ReplicationPhase(ShardIterator originalShardIt, ReplicaRequest replicaRequest, Response finalResponse,
ClusterStateObserver observer, ShardRouting originalPrimaryShard,
InternalRequest internalRequest, ActionListener<Response> listener, Releasable indexShardReference) {
this.replicaRequest = replicaRequest;
this.listener = listener;
this.finalResponse = finalResponse;
this.originalPrimaryShard = originalPrimaryShard;
this.observer = observer;
indexMetaData = observer.observedState().metaData().index(internalRequest.concreteIndex());
this.indexShardReference = indexShardReference;
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();
int numberOfUnassignedOrShadowReplicas = 0;
int numberOfPendingShardInstances = 0;
if (observer.observedState() != newState) {
observer.reset(newState);
shardIt = shards(newState, internalRequest);
while ((shard = shardIt.nextOrNull()) != null) {
if (shard.primary()) {
if (originalPrimaryShard.currentNodeId().equals(shard.currentNodeId()) == false) {
// there is a new primary, we'll have to replicate to it.
numberOfPendingShardInstances++;
}
if (shard.relocating()) {
numberOfPendingShardInstances++;
}
} else if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.settings())) {
// If the replicas use shadow replicas, there is no reason to
// perform the action on the replica, so skip it and
// immediately return
// this delays mapping updates on replicas because they have
// to wait until they get the new mapping through the cluster
// state, which is why we recommend pre-defined mappings for
// indices using shadow replicas
numberOfUnassignedOrShadowReplicas++;
} else if (shard.unassigned()) {
numberOfUnassignedOrShadowReplicas++;
} else if (shard.relocating()) {
// we need to send to two copies
numberOfPendingShardInstances += 2;
} else {
numberOfPendingShardInstances++;
}
}
internalRequest.request().setCanHaveDuplicates(); // safe side, cluster state changed, we might have dups
} else {
shardIt = originalShardIt;
shardIt.reset();
while ((shard = shardIt.nextOrNull()) != null) {
if (shard.state() != ShardRoutingState.STARTED) {
replicaRequest.setCanHaveDuplicates();
}
if (shard.unassigned()) {
numberOfUnassignedOrShadowReplicas++;
} else if (shard.primary()) {
if (shard.relocating()) {
// we have to replicate to the other copy
numberOfPendingShardInstances += 1;
}
} else if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.settings())) {
// If the replicas use shadow replicas, there is no reason to
// perform the action on the replica, so skip it and
// immediately return
// this delays mapping updates on replicas because they have
// to wait until they get the new mapping through the cluster
// state, which is why we recommend pre-defined mappings for
// indices using shadow replicas
numberOfUnassignedOrShadowReplicas++;
} else if (shard.relocating()) {
// we need to send to two copies
numberOfPendingShardInstances += 2;
} else {
numberOfPendingShardInstances++;
}
}
}
// one for the primary already done
this.totalShards = 1 + numberOfPendingShardInstances + numberOfUnassignedOrShadowReplicas;
this.pending = new AtomicInteger(numberOfPendingShardInstances);
}
/**
* total shard copies
*/
int totalShards() {
return totalShards;
}
/**
* total successful operations so far
*/
int successful() {
return success.get();
}
/**
* number of pending operations
*/
int pending() {
return pending.get();
}
@Override
public void onFailure(Throwable t) {
logger.error("unexpected error while replicating for action [{}]. shard [{}]. ", t, actionName, shardIt.shardId());
forceFinishAsFailed(t);
}
/**
* start sending current requests to replicas
*/
@Override
protected void doRun() {
if (pending.get() == 0) {
doFinish();
return;
}
ShardRouting shard;
shardIt.reset(); // reset the iterator
while ((shard = shardIt.nextOrNull()) != null) {
// if its unassigned, nothing to do here...
if (shard.unassigned()) {
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 (shard.primary()) {
if (originalPrimaryShard.currentNodeId().equals(shard.currentNodeId()) == false) {
// there is a new primary, we'll have to replicate to it.
performOnReplica(shard, shard.currentNodeId());
}
if (shard.relocating()) {
performOnReplica(shard, shard.relocatingNodeId());
}
} else if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.settings()) == false) {
performOnReplica(shard, shard.currentNodeId());
if (shard.relocating()) {
performOnReplica(shard, shard.relocatingNodeId());
}
}
}
}
/**
* send operation to the given node or perform it if local
*/
void performOnReplica(final ShardRouting shard, final String nodeId) {
// 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)) {
onReplicaFailure(nodeId, null);
return;
}
replicaRequest.internalShardId = shardIt.shardId();
if (!nodeId.equals(observer.observedState().nodes().localNodeId())) {
final DiscoveryNode node = observer.observedState().nodes().get(nodeId);
transportService.sendRequest(node, transportReplicaAction, replicaRequest,
transportOptions, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) {
@Override
public void handleResponse(TransportResponse.Empty vResponse) {
onReplicaSuccess();
}
@Override
public void handleException(TransportException exp) {
onReplicaFailure(nodeId, exp);
logger.trace("[{}] transport failure during replica request [{}] ", exp, node, replicaRequest);
if (ignoreReplicaException(exp) == false) {
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) + "]");
}
}
});
} else {
if (replicaRequest.operationThreaded()) {
try {
threadPool.executor(executor).execute(new AbstractRunnable() {
@Override
protected void doRun() {
try {
shardOperationOnReplica(shard.shardId(), replicaRequest);
onReplicaSuccess();
} catch (Throwable e) {
onReplicaFailure(nodeId, e);
failReplicaIfNeeded(shard.index(), shard.id(), e);
}
}
// we must never reject on because of thread pool capacity on replicas
@Override
public boolean isForceExecution() {
return true;
}
@Override
public void onFailure(Throwable t) {
onReplicaFailure(nodeId, t);
}
});
} catch (Throwable e) {
failReplicaIfNeeded(shard.index(), shard.id(), e);
onReplicaFailure(nodeId, e);
}
} else {
try {
shardOperationOnReplica(shard.shardId(), replicaRequest);
onReplicaSuccess();
} catch (Throwable e) {
failReplicaIfNeeded(shard.index(), shard.id(), e);
onReplicaFailure(nodeId, e);
}
}
}
}
void onReplicaFailure(String nodeId, @Nullable Throwable e) {
// Only version conflict should be ignored from being put into the _shards header?
if (e != null && ignoreReplicaException(e) == false) {
shardReplicaFailures.put(nodeId, e);
}
decPendingAndFinishIfNeeded();
}
void onReplicaSuccess() {
success.incrementAndGet();
decPendingAndFinishIfNeeded();
}
private void decPendingAndFinishIfNeeded() {
if (pending.decrementAndGet() <= 0) {
doFinish();
}
}
private void forceFinishAsFailed(Throwable t) {
if (finished.compareAndSet(false, true)) {
Releasables.close(indexShardReference);
listener.onFailure(t);
}
}
private void doFinish() {
if (finished.compareAndSet(false, true)) {
Releasables.close(indexShardReference);
final ShardId shardId = shardIt.shardId();
final ActionWriteResponse.ShardInfo.Failure[] failuresArray;
if (!shardReplicaFailures.isEmpty()) {
int slot = 0;
failuresArray = new ActionWriteResponse.ShardInfo.Failure[shardReplicaFailures.size()];
for (Map.Entry<String, Throwable> entry : shardReplicaFailures.entrySet()) {
RestStatus restStatus = ExceptionsHelper.status(entry.getValue());
failuresArray[slot++] = new ActionWriteResponse.ShardInfo.Failure(
shardId.getIndex(), shardId.getId(), entry.getKey(), entry.getValue(), restStatus, false
);
}
} else {
failuresArray = ActionWriteResponse.EMPTY;
}
finalResponse.setShardInfo(new ActionWriteResponse.ShardInfo(
totalShards,
success.get(),
failuresArray
)
);
listener.onResponse(finalResponse);