summaryrefslogtreecommitdiff
path: root/core/src/test/java/org/elasticsearch/index/replication/ESIndexLevelReplicationTestCase.java
blob: 7962f23caf0c089726bec2f52d2a483959ef8722 (plain)
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
/*
 * 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.index.replication;

import org.apache.lucene.store.AlreadyClosedException;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.bulk.BulkItemRequest;
import org.elasticsearch.action.bulk.BulkItemResponse;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.bulk.BulkShardResponse;
import org.elasticsearch.action.bulk.TransportShardBulkAction;
import org.elasticsearch.action.bulk.TransportShardBulkActionTests;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.resync.ResyncReplicationRequest;
import org.elasticsearch.action.resync.ResyncReplicationResponse;
import org.elasticsearch.action.resync.TransportResyncReplicationAction;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.replication.ReplicationOperation;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportReplicationAction.ReplicaResponse;
import org.elasticsearch.action.support.replication.TransportWriteAction;
import org.elasticsearch.action.support.replication.TransportWriteActionTestHelper;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RecoverySource;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.collect.Iterators;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.seqno.GlobalCheckpointSyncAction;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardTestCase;
import org.elasticsearch.index.shard.PrimaryReplicaSyncer;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.indices.recovery.RecoveryTarget;
import org.elasticsearch.tasks.TaskManager;
import org.elasticsearch.threadpool.ThreadPool;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.Future;
import java.util.concurrent.FutureTask;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;

import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;

public abstract class ESIndexLevelReplicationTestCase extends IndexShardTestCase {

    protected final Index index = new Index("test", "uuid");
    private final ShardId shardId = new ShardId(index, 0);
    private final Map<String, String> indexMapping = Collections.singletonMap("type", "{ \"type\": {} }");

    protected ReplicationGroup createGroup(int replicas) throws IOException {
        IndexMetaData metaData = buildIndexMetaData(replicas);
        return new ReplicationGroup(metaData);
    }

    protected IndexMetaData buildIndexMetaData(int replicas) throws IOException {
        return buildIndexMetaData(replicas, indexMapping);
    }

    protected IndexMetaData buildIndexMetaData(int replicas, Map<String, String> mappings) throws IOException {
        Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
            .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, replicas)
            .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
            .build();
        IndexMetaData.Builder metaData = IndexMetaData.builder(index.getName())
            .settings(settings)
            .primaryTerm(0, randomIntBetween(1, 100));
        for (Map.Entry<String, String> typeMapping : mappings.entrySet()) {
            metaData.putMapping(typeMapping.getKey(), typeMapping.getValue());
        }
        return metaData.build();
    }

    protected DiscoveryNode getDiscoveryNode(String id) {
        return new DiscoveryNode(id, id, buildNewFakeTransportAddress(), Collections.emptyMap(),
            Collections.singleton(DiscoveryNode.Role.DATA), Version.CURRENT);
    }

    protected class ReplicationGroup implements AutoCloseable, Iterable<IndexShard> {
        private long clusterStateVersion;
        private IndexShard primary;
        private IndexMetaData indexMetaData;
        private final List<IndexShard> replicas;
        private final AtomicInteger replicaId = new AtomicInteger();
        private final AtomicInteger docId = new AtomicInteger();
        boolean closed = false;
        private final PrimaryReplicaSyncer primaryReplicaSyncer = new PrimaryReplicaSyncer(Settings.EMPTY, new TaskManager(Settings.EMPTY),
            (request, parentTask, primaryAllocationId, listener) -> {
                try {
                    new ResyncAction(request, listener, ReplicationGroup.this).execute();
                } catch (Exception e) {
                    throw new AssertionError(e);
                }
            });

        ReplicationGroup(final IndexMetaData indexMetaData) throws IOException {
            final ShardRouting primaryRouting = this.createShardRouting("s0", true);
            primary = newShard(primaryRouting, indexMetaData, null, getEngineFactory(primaryRouting));
            replicas = new ArrayList<>();
            this.indexMetaData = indexMetaData;
            clusterStateVersion = 1;
            updateAllocationIDsOnPrimary();
            for (int i = 0; i < indexMetaData.getNumberOfReplicas(); i++) {
                addReplica();
            }
        }

        private ShardRouting createShardRouting(String nodeId, boolean primary) {
            return TestShardRouting.newShardRouting(shardId, nodeId, primary, ShardRoutingState.INITIALIZING,
                primary ? RecoverySource.StoreRecoverySource.EMPTY_STORE_INSTANCE : RecoverySource.PeerRecoverySource.INSTANCE);
        }

        protected EngineFactory getEngineFactory(ShardRouting routing) {
            return null;
        }

        public int indexDocs(final int numOfDoc) throws Exception {
            for (int doc = 0; doc < numOfDoc; doc++) {
                final IndexRequest indexRequest = new IndexRequest(index.getName(), "type", Integer.toString(docId.incrementAndGet()))
                        .source("{}", XContentType.JSON);
                final BulkItemResponse response = index(indexRequest);
                if (response.isFailed()) {
                    throw response.getFailure().getCause();
                } else {
                    assertEquals(DocWriteResponse.Result.CREATED, response.getResponse().getResult());
                }
            }
            return numOfDoc;
        }

        public int appendDocs(final int numOfDoc) throws Exception {
            for (int doc = 0; doc < numOfDoc; doc++) {
                final IndexRequest indexRequest = new IndexRequest(index.getName(), "type").source("{}", XContentType.JSON);
                final BulkItemResponse response = index(indexRequest);
                if (response.isFailed()) {
                    throw response.getFailure().getCause();
                } else if (response.isFailed() == false) {
                    assertEquals(DocWriteResponse.Result.CREATED, response.getResponse().getResult());
                }
            }
            return numOfDoc;
        }

        public BulkItemResponse index(IndexRequest indexRequest) throws Exception {
            PlainActionFuture<BulkItemResponse> listener = new PlainActionFuture<>();
            final ActionListener<BulkShardResponse> wrapBulkListener = ActionListener.wrap(
                    bulkShardResponse -> listener.onResponse(bulkShardResponse.getResponses()[0]),
                    listener::onFailure);
            BulkItemRequest[] items = new BulkItemRequest[1];
            items[0] = new BulkItemRequest(0, indexRequest);
            BulkShardRequest request = new BulkShardRequest(shardId, indexRequest.getRefreshPolicy(), items);
            new IndexingAction(request, wrapBulkListener, this).execute();
            return listener.get();
        }

        public synchronized void startAll() throws IOException {
            startReplicas(replicas.size());
        }

        public synchronized int startReplicas(int numOfReplicasToStart) throws IOException {
            if (primary.routingEntry().initializing()) {
                startPrimary();
            }
            int started = 0;
            for (IndexShard replicaShard : replicas) {
                if (replicaShard.routingEntry().initializing()) {
                    recoverReplica(replicaShard);
                    started++;
                    if (started > numOfReplicasToStart) {
                        break;
                    }
                }
            }
            return started;
        }

        public void startPrimary() throws IOException {
            final DiscoveryNode pNode = getDiscoveryNode(primary.routingEntry().currentNodeId());
            primary.markAsRecovering("store", new RecoveryState(primary.routingEntry(), pNode, null));
            primary.recoverFromStore();
            primary.updateRoutingEntry(ShardRoutingHelper.moveToStarted(primary.routingEntry()));
            clusterStateVersion++;
            updateAllocationIDsOnPrimary();
            for (final IndexShard replica : replicas) {
                recoverReplica(replica);
            }
        }

        public IndexShard addReplica() throws IOException {
            final ShardRouting replicaRouting = createShardRouting("s" + replicaId.incrementAndGet(), false);
            final IndexShard replica =
                newShard(replicaRouting, indexMetaData, null, getEngineFactory(replicaRouting));
            addReplica(replica);
            return replica;
        }

        public synchronized void addReplica(IndexShard replica) {
            assert shardRoutings().stream()
                .filter(shardRouting -> shardRouting.isSameAllocation(replica.routingEntry())).findFirst().isPresent() == false :
                "replica with aId [" + replica.routingEntry().allocationId() + "] already exists";
            replicas.add(replica);
            clusterStateVersion++;
            updateAllocationIDsOnPrimary();
        }


        public synchronized IndexShard addReplicaWithExistingPath(final ShardPath shardPath, final String nodeId) throws IOException {
            final ShardRouting shardRouting = TestShardRouting.newShardRouting(
                shardId,
                nodeId,
                false, ShardRoutingState.INITIALIZING,
                RecoverySource.PeerRecoverySource.INSTANCE);

            final IndexShard newReplica = newShard(shardRouting, shardPath, indexMetaData, null,
                    getEngineFactory(shardRouting));
            replicas.add(newReplica);
            clusterStateVersion++;
            updateAllocationIDsOnPrimary();
            return newReplica;
        }

        public synchronized List<IndexShard> getReplicas() {
            return Collections.unmodifiableList(replicas);
        }

        /**
         * promotes the specific replica as the new primary
         */
        public synchronized Future<PrimaryReplicaSyncer.ResyncTask> promoteReplicaToPrimary(IndexShard replica) throws IOException {
            final long newTerm = indexMetaData.primaryTerm(shardId.id()) + 1;
            IndexMetaData.Builder newMetaData = IndexMetaData.builder(indexMetaData).primaryTerm(shardId.id(), newTerm);
            indexMetaData = newMetaData.build();
            assertTrue(replicas.remove(replica));
            closeShards(primary);
            primary = replica;
            primary.updateRoutingEntry(replica.routingEntry().moveActiveReplicaToPrimary());

            PlainActionFuture<PrimaryReplicaSyncer.ResyncTask> fut = new PlainActionFuture<>();
            primary.updatePrimaryTerm(newTerm, (shard, listener) -> primaryReplicaSyncer.resync(shard,
                new ActionListener<PrimaryReplicaSyncer.ResyncTask>() {
                    @Override
                    public void onResponse(PrimaryReplicaSyncer.ResyncTask resyncTask) {
                        listener.onResponse(resyncTask);
                        fut.onResponse(resyncTask);
                    }

                    @Override
                    public void onFailure(Exception e) {
                        listener.onFailure(e);
                        fut.onFailure(e);
                    }
                }));
            clusterStateVersion++;
            updateAllocationIDsOnPrimary();
            return fut;
        }

        synchronized boolean removeReplica(IndexShard replica) {
            final boolean removed = replicas.remove(replica);
            if (removed) {
                clusterStateVersion++;
                updateAllocationIDsOnPrimary();
            }
            return removed;
        }

        public void recoverReplica(IndexShard replica) throws IOException {
            recoverReplica(replica, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener, version -> {}));
        }

        public void recoverReplica(IndexShard replica, BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier)
            throws IOException {
            recoverReplica(replica, targetSupplier, true);
        }

        public void recoverReplica(
            IndexShard replica,
            BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier,
            boolean markAsRecovering) throws IOException {
            ESIndexLevelReplicationTestCase.this.recoverReplica(replica, primary, targetSupplier, markAsRecovering);
            clusterStateVersion++;
            updateAllocationIDsOnPrimary();
        }

        public synchronized DiscoveryNode getPrimaryNode() {
            return getDiscoveryNode(primary.routingEntry().currentNodeId());
        }

        public Future<Void> asyncRecoverReplica(
                final IndexShard replica, final BiFunction<IndexShard, DiscoveryNode, RecoveryTarget> targetSupplier) throws IOException {
            final FutureTask<Void> task = new FutureTask<>(() -> {
                recoverReplica(replica, targetSupplier);
                return null;
            });
            threadPool.generic().execute(task);
            return task;
        }

        public synchronized void assertAllEqual(int expectedCount) throws IOException {
            Set<String> primaryIds = getShardDocUIDs(primary);
            assertThat(primaryIds.size(), equalTo(expectedCount));
            for (IndexShard replica : replicas) {
                Set<String> replicaIds = getShardDocUIDs(replica);
                Set<String> temp = new HashSet<>(primaryIds);
                temp.removeAll(replicaIds);
                assertThat(replica.routingEntry() + " is missing docs", temp, empty());
                temp = new HashSet<>(replicaIds);
                temp.removeAll(primaryIds);
                assertThat(replica.routingEntry() + " has extra docs", temp, empty());
            }
        }

        public synchronized void refresh(String source) {
            for (IndexShard shard : this) {
                shard.refresh(source);
            }
        }

        public synchronized void flush() {
            final FlushRequest request = new FlushRequest();
            for (IndexShard shard : this) {
                shard.flush(request);
            }
        }

        public synchronized List<ShardRouting> shardRoutings() {
            return StreamSupport.stream(this.spliterator(), false).map(IndexShard::routingEntry).collect(Collectors.toList());
        }

        @Override
        public synchronized void close() throws Exception {
            if (closed == false) {
                closed = true;
                closeShards(this);
            } else {
                throw new AlreadyClosedException("too bad");
            }
        }

        @Override
        public Iterator<IndexShard> iterator() {
            return Iterators.concat(replicas.iterator(), Collections.singleton(primary).iterator());
        }

        public IndexShard getPrimary() {
            return primary;
        }

        public void syncGlobalCheckpoint() {
            PlainActionFuture<ReplicationResponse> listener = new PlainActionFuture<>();
            try {
                new GlobalCheckpointSync(listener, this).execute();
                listener.get();
            } catch (Exception e) {
                throw new AssertionError(e);
            }
        }

        private void updateAllocationIDsOnPrimary() {
            Set<String> active = new HashSet<>();
            Set<String> initializing = new HashSet<>();
            for (ShardRouting shard: shardRoutings()) {
                if (shard.active()) {
                    active.add(shard.allocationId().getId());
                } else {
                    initializing.add(shard.allocationId().getId());
                }
            }
            primary.updateAllocationIdsFromMaster(clusterStateVersion, active, initializing);
        }
    }

    abstract class ReplicationAction<Request extends ReplicationRequest<Request>,
        ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
        Response extends ReplicationResponse> {
        private final Request request;
        private ActionListener<Response> listener;
        private final ReplicationGroup replicationGroup;
        private final String opType;

        ReplicationAction(Request request, ActionListener<Response> listener, ReplicationGroup group, String opType) {
            this.request = request;
            this.listener = listener;
            this.replicationGroup = group;
            this.opType = opType;
        }

        public void execute() {
            try {
                new ReplicationOperation<Request, ReplicaRequest, PrimaryResult>(request, new PrimaryRef(),
                    new ActionListener<PrimaryResult>() {
                        @Override
                        public void onResponse(PrimaryResult result) {
                            result.respond(listener);
                        }

                        @Override
                        public void onFailure(Exception e) {
                            listener.onFailure(e);
                        }
                    }, new ReplicasRef(), () -> null, logger, opType) {

                    @Override
                    protected List<ShardRouting> getShards(ShardId shardId, ClusterState state) {
                        return replicationGroup.shardRoutings();
                    }

                    @Override
                    protected String checkActiveShardCount() {
                        return null;
                    }

                    @Override
                    protected Set<String> getInSyncAllocationIds(ShardId shardId, ClusterState clusterState) {
                        return replicationGroup.shardRoutings().stream().filter(ShardRouting::active).map(r -> r.allocationId().getId())
                            .collect(Collectors.toSet());
                    }
                }.execute();
            } catch (Exception e) {
                listener.onFailure(e);
            }
        }

        protected abstract PrimaryResult performOnPrimary(IndexShard primary, Request request) throws Exception;

        protected abstract void performOnReplica(ReplicaRequest request, IndexShard replica) throws Exception;

        class PrimaryRef implements ReplicationOperation.Primary<Request, ReplicaRequest, PrimaryResult> {

            @Override
            public ShardRouting routingEntry() {
                return replicationGroup.primary.routingEntry();
            }

            @Override
            public void failShard(String message, Exception exception) {
                throw new UnsupportedOperationException();
            }

            @Override
            public PrimaryResult perform(Request request) throws Exception {
                PrimaryResult response = performOnPrimary(replicationGroup.primary, request);
                response.replicaRequest().primaryTerm(replicationGroup.primary.getPrimaryTerm());
                return response;
            }

            @Override
            public void updateLocalCheckpointForShard(String allocationId, long checkpoint) {
                replicationGroup.getPrimary().updateLocalCheckpointForShard(allocationId, checkpoint);
            }

            @Override
            public long localCheckpoint() {
                return replicationGroup.getPrimary().getLocalCheckpoint();
            }

            @Override
            public long globalCheckpoint() {
                return replicationGroup.getPrimary().getGlobalCheckpoint();
            }

        }

        class ReplicasRef implements ReplicationOperation.Replicas<ReplicaRequest> {

            @Override
            public void performOn(
                final ShardRouting replicaRouting,
                final ReplicaRequest request,
                final long globalCheckpoint,
                final ActionListener<ReplicationOperation.ReplicaResponse> listener) {
                IndexShard replica = replicationGroup.replicas.stream()
                        .filter(s -> replicaRouting.isSameAllocation(s.routingEntry())).findFirst().get();
                replica.acquireReplicaOperationPermit(
                        request.primaryTerm(),
                        new ActionListener<Releasable>() {
                            @Override
                            public void onResponse(Releasable releasable) {
                                try {
                                    replica.updateGlobalCheckpointOnReplica(globalCheckpoint);
                                    performOnReplica(request, replica);
                                    releasable.close();
                                    listener.onResponse(
                                            new ReplicaResponse(
                                                    replica.routingEntry().allocationId().getId(), replica.getLocalCheckpoint()));
                                } catch (final Exception e) {
                                    Releasables.closeWhileHandlingException(releasable);
                                    listener.onFailure(e);
                                }
                            }

                            @Override
                            public void onFailure(Exception e) {
                                listener.onFailure(e);
                            }
                        },
                        ThreadPool.Names.INDEX);
            }

            @Override
            public void failShardIfNeeded(ShardRouting replica, long primaryTerm, String message, Exception exception,
                                          Runnable onSuccess, Consumer<Exception> onPrimaryDemoted,
                                          Consumer<Exception> onIgnoredFailure) {
                throw new UnsupportedOperationException();
            }

            @Override
            public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, long primaryTerm, Runnable onSuccess,
                                                     Consumer<Exception> onPrimaryDemoted, Consumer<Exception> onIgnoredFailure) {
                throw new UnsupportedOperationException();
            }
        }

        class PrimaryResult implements ReplicationOperation.PrimaryResult<ReplicaRequest> {
            final ReplicaRequest replicaRequest;
            final Response finalResponse;

            PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) {
                this.replicaRequest = replicaRequest;
                this.finalResponse = finalResponse;
            }

            @Override
            public ReplicaRequest replicaRequest() {
                return replicaRequest;
            }

            @Override
            public void setShardInfo(ReplicationResponse.ShardInfo shardInfo) {
                finalResponse.setShardInfo(shardInfo);
            }

            public void respond(ActionListener<Response> listener) {
                listener.onResponse(finalResponse);
            }
        }

    }

    class IndexingAction extends ReplicationAction<BulkShardRequest, BulkShardRequest, BulkShardResponse> {

        IndexingAction(BulkShardRequest request, ActionListener<BulkShardResponse> listener, ReplicationGroup replicationGroup) {
            super(request, listener, replicationGroup, "indexing");
        }

        @Override
        protected PrimaryResult performOnPrimary(IndexShard primary, BulkShardRequest request) throws Exception {
            final TransportWriteAction.WritePrimaryResult<BulkShardRequest, BulkShardResponse> result = executeShardBulkOnPrimary(primary, request);
            return new PrimaryResult(result.replicaRequest(), result.finalResponseIfSuccessful);
        }

        @Override
        protected void performOnReplica(BulkShardRequest request, IndexShard replica) throws Exception {
            executeShardBulkOnReplica(replica, request);
        }
    }

    private TransportWriteAction.WritePrimaryResult<BulkShardRequest, BulkShardResponse> executeShardBulkOnPrimary(IndexShard primary, BulkShardRequest request) throws Exception {
        for (BulkItemRequest itemRequest : request.items()) {
            if (itemRequest.request() instanceof IndexRequest) {
                ((IndexRequest) itemRequest.request()).process(null, index.getName());
            }
        }
        final TransportWriteAction.WritePrimaryResult<BulkShardRequest, BulkShardResponse> result =
                TransportShardBulkAction.performOnPrimary(request, primary, null,
                System::currentTimeMillis, new TransportShardBulkActionTests.NoopMappingUpdatePerformer());
        request.primaryTerm(primary.getPrimaryTerm());
        TransportWriteActionTestHelper.performPostWriteActions(primary, request, result.location, logger);
        return result;
    }

    private void executeShardBulkOnReplica(IndexShard replica, BulkShardRequest request) throws Exception {
        final Translog.Location location = TransportShardBulkAction.performOnReplica(request, replica);
        TransportWriteActionTestHelper.performPostWriteActions(replica, request, location, logger);
    }

    /**
     * indexes the given requests on the supplied primary, modifying it for replicas
     */
    BulkShardRequest indexOnPrimary(IndexRequest request, IndexShard primary) throws Exception {
        final BulkItemRequest bulkItemRequest = new BulkItemRequest(0, request);
        BulkItemRequest[] bulkItemRequests = new BulkItemRequest[1];
        bulkItemRequests[0] = bulkItemRequest;
        final BulkShardRequest bulkShardRequest = new BulkShardRequest(shardId, request.getRefreshPolicy(), bulkItemRequests);
        final TransportWriteAction.WritePrimaryResult<BulkShardRequest, BulkShardResponse> result =
                executeShardBulkOnPrimary(primary, bulkShardRequest);
        return result.replicaRequest();
    }

    /**
     * indexes the given requests on the supplied replica shard
     */
    void indexOnReplica(BulkShardRequest request, IndexShard replica) throws Exception {
        executeShardBulkOnReplica(replica, request);
    }

    class GlobalCheckpointSync extends ReplicationAction<
            GlobalCheckpointSyncAction.Request,
            GlobalCheckpointSyncAction.Request,
            ReplicationResponse> {

        GlobalCheckpointSync(final ActionListener<ReplicationResponse> listener, final ReplicationGroup replicationGroup) {
            super(
                    new GlobalCheckpointSyncAction.Request(replicationGroup.getPrimary().shardId()),
                    listener,
                    replicationGroup,
                    "global_checkpoint_sync");
        }

        @Override
        protected PrimaryResult performOnPrimary(
                final IndexShard primary, final GlobalCheckpointSyncAction.Request request) throws Exception {
            primary.getTranslog().sync();
            return new PrimaryResult(request, new ReplicationResponse());
        }

        @Override
        protected void performOnReplica(final GlobalCheckpointSyncAction.Request request, final IndexShard replica) throws IOException {
            replica.getTranslog().sync();
        }
    }

    class ResyncAction extends ReplicationAction<ResyncReplicationRequest, ResyncReplicationRequest, ResyncReplicationResponse> {

        ResyncAction(ResyncReplicationRequest request, ActionListener<ResyncReplicationResponse> listener, ReplicationGroup replicationGroup) {
            super(request, listener, replicationGroup, "resync");
        }

        @Override
        protected PrimaryResult performOnPrimary(IndexShard primary, ResyncReplicationRequest request) throws Exception {
            final TransportWriteAction.WritePrimaryResult<ResyncReplicationRequest, ResyncReplicationResponse> result =
                executeResyncOnPrimary(primary, request);
            return new PrimaryResult(result.replicaRequest(), result.finalResponseIfSuccessful);
        }

        @Override
        protected void performOnReplica(ResyncReplicationRequest request, IndexShard replica) throws Exception {
            executeResyncOnReplica(replica, request);
        }
    }

    private TransportWriteAction.WritePrimaryResult<ResyncReplicationRequest, ResyncReplicationResponse> executeResyncOnPrimary(
        IndexShard primary, ResyncReplicationRequest request) throws Exception {
        final TransportWriteAction.WritePrimaryResult<ResyncReplicationRequest, ResyncReplicationResponse> result =
            new TransportWriteAction.WritePrimaryResult<>(TransportResyncReplicationAction.performOnPrimary(request, primary),
                new ResyncReplicationResponse(), null, null, primary, logger);
        request.primaryTerm(primary.getPrimaryTerm());
        TransportWriteActionTestHelper.performPostWriteActions(primary, request, result.location, logger);
        return result;
    }

    private void executeResyncOnReplica(IndexShard replica, ResyncReplicationRequest request) throws Exception {
        final Translog.Location location = TransportResyncReplicationAction.performOnReplica(request, replica);
        TransportWriteActionTestHelper.performPostWriteActions(replica, request, location, logger);
    }
}