summaryrefslogtreecommitdiff
path: root/core/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java
blob: 87608dedd1e9b10effcef5e6314c630cc7d9c5c0 (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
/*
 * 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.logging.log4j.Logger;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.lucene.uid.Versions;
import org.elasticsearch.common.xcontent.XContentType;
import org.elasticsearch.index.VersionType;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineConfig;
import org.elasticsearch.index.engine.EngineFactory;
import org.elasticsearch.index.engine.InternalEngineTests;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.PrimaryReplicaSyncer;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.recovery.PeerRecoveryTargetService;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.indices.recovery.RecoveryTarget;
import org.elasticsearch.test.junit.annotations.TestLogging;

import java.io.IOException;
import java.util.ArrayList;
import java.util.EnumSet;
import java.util.List;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Future;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;

import static org.hamcrest.Matchers.anyOf;
import static org.hamcrest.Matchers.empty;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.lessThan;
import static org.hamcrest.Matchers.not;

public class RecoveryDuringReplicationTests extends ESIndexLevelReplicationTestCase {

    public void testIndexingDuringFileRecovery() throws Exception {
        try (ReplicationGroup shards = createGroup(randomInt(1))) {
            shards.startAll();
            int docs = shards.indexDocs(randomInt(50));
            shards.flush();
            IndexShard replica = shards.addReplica();
            final CountDownLatch recoveryBlocked = new CountDownLatch(1);
            final CountDownLatch releaseRecovery = new CountDownLatch(1);
            final RecoveryState.Stage blockOnStage = randomFrom(BlockingTarget.SUPPORTED_STAGES);
            final Future<Void> recoveryFuture = shards.asyncRecoverReplica(replica, (indexShard, node) ->
                new BlockingTarget(blockOnStage, recoveryBlocked, releaseRecovery, indexShard, node, recoveryListener, logger));

            recoveryBlocked.await();
            docs += shards.indexDocs(randomInt(20));
            releaseRecovery.countDown();
            recoveryFuture.get();

            shards.assertAllEqual(docs);
        }
    }

    public void testRecoveryOfDisconnectedReplica() throws Exception {
        try (ReplicationGroup shards = createGroup(1)) {
            shards.startAll();
            int docs = shards.indexDocs(randomInt(50));
            shards.flush();
            final IndexShard originalReplica = shards.getReplicas().get(0);
            long replicaCommittedLocalCheckpoint = docs - 1;
            boolean replicaHasDocsSinceLastFlushedCheckpoint = false;
            for (int i = 0; i < randomInt(2); i++) {
                final int indexedDocs = shards.indexDocs(randomInt(5));
                docs += indexedDocs;
                if (indexedDocs > 0) {
                    replicaHasDocsSinceLastFlushedCheckpoint = true;
                }

                final boolean flush = randomBoolean();
                if (flush) {
                    originalReplica.flush(new FlushRequest());
                    replicaHasDocsSinceLastFlushedCheckpoint = false;
                    replicaCommittedLocalCheckpoint = docs - 1;
                }
            }

            // simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
            shards.syncGlobalCheckpoint();

            shards.removeReplica(originalReplica);

            final int missingOnReplica = shards.indexDocs(randomInt(5));
            docs += missingOnReplica;
            replicaHasDocsSinceLastFlushedCheckpoint |= missingOnReplica > 0;

            final boolean translogTrimmed;
            if (randomBoolean()) {
                shards.flush();
                translogTrimmed = randomBoolean();
                if (translogTrimmed) {
                    final Translog translog = shards.getPrimary().getTranslog();
                    translog.getDeletionPolicy().setRetentionAgeInMillis(0);
                    translog.trimUnreferencedReaders();
                }
            } else {
                translogTrimmed = false;
            }
            originalReplica.close("disconnected", false);
            IOUtils.close(originalReplica.store());
            final IndexShard recoveredReplica =
                shards.addReplicaWithExistingPath(originalReplica.shardPath(), originalReplica.routingEntry().currentNodeId());
            shards.recoverReplica(recoveredReplica);
            if (translogTrimmed && replicaHasDocsSinceLastFlushedCheckpoint) {
                // replica has something to catch up with, but since we trimmed the primary translog, we should fall back to full recovery
                assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), not(empty()));
            } else {
                assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), empty());
                assertThat(
                    recoveredReplica.recoveryState().getTranslog().recoveredOperations(),
                    equalTo(Math.toIntExact(docs - (replicaCommittedLocalCheckpoint + 1))));
            }

            docs += shards.indexDocs(randomInt(5));

            shards.assertAllEqual(docs);
        }
    }

    /*
     * Simulate a scenario with two replicas where one of the replicas receives an extra document, the other replica is promoted on primary
     * failure, the receiving replica misses the primary/replica re-sync and then recovers from the primary. We expect that a
     * sequence-number based recovery is performed and the extra document does not remain after recovery.
     */
    public void testRecoveryToReplicaThatReceivedExtraDocument() throws Exception {
        try (ReplicationGroup shards = createGroup(2)) {
            shards.startAll();
            final int docs = randomIntBetween(0, 16);
            for (int i = 0; i < docs; i++) {
                shards.index(
                        new IndexRequest("index", "type", Integer.toString(i)).source("{}", XContentType.JSON));
            }

            shards.flush();
            shards.syncGlobalCheckpoint();

            final IndexShard oldPrimary = shards.getPrimary();
            final IndexShard promotedReplica = shards.getReplicas().get(0);
            final IndexShard remainingReplica = shards.getReplicas().get(1);
            // slip the extra document into the replica
            remainingReplica.applyIndexOperationOnReplica(
                    remainingReplica.getLocalCheckpoint() + 1,
                    remainingReplica.getPrimaryTerm(),
                    1,
                    VersionType.EXTERNAL,
                    randomNonNegativeLong(),
                    false,
                    SourceToParse.source("index", "type", "replica", new BytesArray("{}"), XContentType.JSON),
                    mapping -> {});
            shards.promoteReplicaToPrimary(promotedReplica);
            oldPrimary.close("demoted", randomBoolean());
            oldPrimary.store().close();
            shards.removeReplica(remainingReplica);
            remainingReplica.close("disconnected", false);
            remainingReplica.store().close();
            // randomly introduce a conflicting document
            final boolean extra = randomBoolean();
            if (extra) {
                promotedReplica.applyIndexOperationOnPrimary(
                        Versions.MATCH_ANY,
                        VersionType.INTERNAL,
                        SourceToParse.source("index", "type", "primary", new BytesArray("{}"), XContentType.JSON),
                        randomNonNegativeLong(),
                        false,
                        mapping -> {
                        });
            }
            final IndexShard recoveredReplica =
                    shards.addReplicaWithExistingPath(remainingReplica.shardPath(), remainingReplica.routingEntry().currentNodeId());
            shards.recoverReplica(recoveredReplica);

            assertThat(recoveredReplica.recoveryState().getIndex().fileDetails(), empty());
            assertThat(recoveredReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(extra ? 1 : 0));

            shards.assertAllEqual(docs + (extra ? 1 : 0));
        }
    }

    @TestLogging("org.elasticsearch.index.shard:TRACE,org.elasticsearch.indices.recovery:TRACE")
    public void testRecoveryAfterPrimaryPromotion() throws Exception {
        try (ReplicationGroup shards = createGroup(2)) {
            shards.startAll();
            int totalDocs = shards.indexDocs(randomInt(10));
            int committedDocs = 0;
            if (randomBoolean()) {
                shards.flush();
                committedDocs = totalDocs;
            }
            // we need some indexing to happen to transfer local checkpoint information to the primary
            // so it can update the global checkpoint and communicate to replicas
            boolean expectSeqNoRecovery = totalDocs > 0;


            final IndexShard oldPrimary = shards.getPrimary();
            final IndexShard newPrimary = shards.getReplicas().get(0);
            final IndexShard replica = shards.getReplicas().get(1);
            if (randomBoolean()) {
                // simulate docs that were inflight when primary failed, these will be rolled back
                final int rollbackDocs = randomIntBetween(1, 5);
                logger.info("--> indexing {} rollback docs", rollbackDocs);
                for (int i = 0; i < rollbackDocs; i++) {
                    final IndexRequest indexRequest = new IndexRequest(index.getName(), "type", "rollback_" + i)
                            .source("{}", XContentType.JSON);
                    final BulkShardRequest bulkShardRequest = indexOnPrimary(indexRequest, oldPrimary);
                    indexOnReplica(bulkShardRequest, replica);
                }
                if (randomBoolean()) {
                    oldPrimary.flush(new FlushRequest(index.getName()));
                    expectSeqNoRecovery = false;
                }
            }

            shards.promoteReplicaToPrimary(newPrimary);
            // index some more
            totalDocs += shards.indexDocs(randomIntBetween(0, 5));

            if (randomBoolean()) {
                newPrimary.flush(new FlushRequest());
            }

            oldPrimary.close("demoted", false);
            oldPrimary.store().close();

            IndexShard newReplica = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId());
            shards.recoverReplica(newReplica);

            if (expectSeqNoRecovery) {
                assertThat(newReplica.recoveryState().getIndex().fileDetails(), empty());
                assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(totalDocs - committedDocs));
            } else {
                assertThat(newReplica.recoveryState().getIndex().fileDetails(), not(empty()));
                assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(totalDocs));
            }

            // roll back the extra ops in the replica
            shards.removeReplica(replica);
            replica.close("resync", false);
            replica.store().close();
            newReplica = shards.addReplicaWithExistingPath(replica.shardPath(), replica.routingEntry().currentNodeId());
            shards.recoverReplica(newReplica);

            shards.assertAllEqual(totalDocs);
        }
    }

    @TestLogging("org.elasticsearch.index.shard:TRACE,org.elasticsearch.action.resync:TRACE")
    public void testResyncAfterPrimaryPromotion() throws Exception {
        // TODO: check translog trimming functionality once it's implemented
        try (ReplicationGroup shards = createGroup(2)) {
            shards.startAll();
            int initialDocs = shards.indexDocs(randomInt(10));
            boolean syncedGlobalCheckPoint = randomBoolean();
            if (syncedGlobalCheckPoint) {
                shards.syncGlobalCheckpoint();
            }

            final IndexShard oldPrimary = shards.getPrimary();
            final IndexShard newPrimary = shards.getReplicas().get(0);
            final IndexShard otherReplica = shards.getReplicas().get(1);

            // simulate docs that were inflight when primary failed
            final int extraDocs = randomIntBetween(0, 5);
            logger.info("--> indexing {} extra docs", extraDocs);
            for (int i = 0; i < extraDocs; i++) {
                final IndexRequest indexRequest = new IndexRequest(index.getName(), "type", "extra_" + i)
                    .source("{}", XContentType.JSON);
                final BulkShardRequest bulkShardRequest = indexOnPrimary(indexRequest, oldPrimary);
                indexOnReplica(bulkShardRequest, newPrimary);
            }
            logger.info("--> resyncing replicas");
            PrimaryReplicaSyncer.ResyncTask task = shards.promoteReplicaToPrimary(newPrimary).get();
            if (syncedGlobalCheckPoint) {
                assertEquals(extraDocs, task.getResyncedOperations());
            } else {
                assertThat(task.getResyncedOperations(), greaterThanOrEqualTo(extraDocs));
            }
            shards.assertAllEqual(initialDocs + extraDocs);
        }
    }

    @TestLogging(
            "_root:DEBUG,"
                    + "org.elasticsearch.action.bulk:TRACE,"
                    + "org.elasticsearch.action.get:TRACE,"
                    + "org.elasticsearch.cluster.service:TRACE,"
                    + "org.elasticsearch.discovery:TRACE,"
                    + "org.elasticsearch.indices.cluster:TRACE,"
                    + "org.elasticsearch.indices.recovery:TRACE,"
                    + "org.elasticsearch.index.seqno:TRACE,"
                    + "org.elasticsearch.index.shard:TRACE")
    public void testWaitForPendingSeqNo() throws Exception {
        IndexMetaData metaData = buildIndexMetaData(1);

        final int pendingDocs = randomIntBetween(1, 5);
        final BlockingEngineFactory primaryEngineFactory = new BlockingEngineFactory();

        try (ReplicationGroup shards = new ReplicationGroup(metaData) {
            @Override
            protected EngineFactory getEngineFactory(ShardRouting routing) {
                if (routing.primary()) {
                    return primaryEngineFactory;
                } else {
                    return null;
                }
            }
        }) {
            shards.startAll();
            int docs = shards.indexDocs(randomIntBetween(1, 10));
            // simulate a background global checkpoint sync at which point we expect the global checkpoint to advance on the replicas
            shards.syncGlobalCheckpoint();
            IndexShard replica = shards.getReplicas().get(0);
            shards.removeReplica(replica);
            closeShards(replica);

            docs += pendingDocs;
            primaryEngineFactory.latchIndexers();
            CountDownLatch pendingDocsDone = new CountDownLatch(pendingDocs);
            for (int i = 0; i < pendingDocs; i++) {
                final String id = "pending_" + i;
                threadPool.generic().submit(() -> {
                    try {
                        shards.index(new IndexRequest(index.getName(), "type", id).source("{}", XContentType.JSON));
                    } catch (Exception e) {
                        throw new AssertionError(e);
                    } finally {
                        pendingDocsDone.countDown();
                    }
                });
            }

            // wait for the pending ops to "hang"
            primaryEngineFactory.awaitIndexersLatch();

            primaryEngineFactory.allowIndexing();
            // index some more
            docs += shards.indexDocs(randomInt(5));

            IndexShard newReplica = shards.addReplicaWithExistingPath(replica.shardPath(), replica.routingEntry().currentNodeId());

            CountDownLatch recoveryStart = new CountDownLatch(1);
            AtomicBoolean preparedForTranslog = new AtomicBoolean(false);
            final Future<Void> recoveryFuture = shards.asyncRecoverReplica(newReplica, (indexShard, node) -> {
                recoveryStart.countDown();
                return new RecoveryTarget(indexShard, node, recoveryListener, l -> {
                }) {
                    @Override
                    public void prepareForTranslogOperations(int totalTranslogOps) throws IOException {
                        preparedForTranslog.set(true);
                        super.prepareForTranslogOperations(totalTranslogOps);
                    }
                };
            });

            recoveryStart.await();

            // index some more
            docs += shards.indexDocs(randomInt(5));

            assertFalse("recovery should wait on pending docs", preparedForTranslog.get());

            primaryEngineFactory.releaseLatchedIndexers();
            pendingDocsDone.await();

            // now recovery can finish
            recoveryFuture.get();

            assertThat(newReplica.recoveryState().getIndex().fileDetails(), empty());
            assertThat(newReplica.recoveryState().getTranslog().recoveredOperations(), equalTo(docs));

            shards.assertAllEqual(docs);
        } finally {
            primaryEngineFactory.close();
        }
    }

    @TestLogging(
            "_root:DEBUG,"
                    + "org.elasticsearch.action.bulk:TRACE,"
                    + "org.elasticsearch.action.get:TRACE,"
                    + "org.elasticsearch.cluster.service:TRACE,"
                    + "org.elasticsearch.discovery:TRACE,"
                    + "org.elasticsearch.indices.cluster:TRACE,"
                    + "org.elasticsearch.indices.recovery:TRACE,"
                    + "org.elasticsearch.index.seqno:TRACE,"
                    + "org.elasticsearch.index.shard:TRACE")
    public void testCheckpointsAndMarkingInSync() throws Exception {
        final IndexMetaData metaData = buildIndexMetaData(0);
        final BlockingEngineFactory replicaEngineFactory = new BlockingEngineFactory();
        try (
                ReplicationGroup shards = new ReplicationGroup(metaData) {
                    @Override
                    protected EngineFactory getEngineFactory(final ShardRouting routing) {
                        if (routing.primary()) {
                            return null;
                        } else {
                            return replicaEngineFactory;
                        }
                    }
                };
                AutoCloseable ignored = replicaEngineFactory // make sure we release indexers before closing
        ) {
            shards.startPrimary();
            final int docs = shards.indexDocs(randomIntBetween(1, 10));
            logger.info("indexed [{}] docs", docs);
            final CountDownLatch pendingDocDone = new CountDownLatch(1);
            final CountDownLatch pendingDocActiveWithExtraDocIndexed = new CountDownLatch(1);
            final CountDownLatch phaseTwoStartLatch = new CountDownLatch(1);
            final IndexShard replica = shards.addReplica();
            final Future<Void> recoveryFuture = shards.asyncRecoverReplica(
                    replica,
                    (indexShard, node) -> new RecoveryTarget(indexShard, node, recoveryListener, l -> {}) {
                        @Override
                        public long indexTranslogOperations(final List<Translog.Operation> operations, final int totalTranslogOps)
                             throws IOException {
                            // index a doc which is not part of the snapshot, but also does not complete on replica
                            replicaEngineFactory.latchIndexers();
                            threadPool.generic().submit(() -> {
                                try {
                                    shards.index(new IndexRequest(index.getName(), "type", "pending").source("{}", XContentType.JSON));
                                } catch (final Exception e) {
                                    throw new RuntimeException(e);
                                } finally {
                                    pendingDocDone.countDown();
                                }
                            });
                            try {
                                // the pending doc is latched in the engine
                                replicaEngineFactory.awaitIndexersLatch();
                                // unblock indexing for the next doc
                                replicaEngineFactory.allowIndexing();
                                shards.index(new IndexRequest(index.getName(), "type", "completed").source("{}", XContentType.JSON));
                                pendingDocActiveWithExtraDocIndexed.countDown();
                            } catch (final Exception e) {
                                throw new AssertionError(e);
                            }
                            try {
                                phaseTwoStartLatch.await();
                            } catch (InterruptedException e) {
                                throw new AssertionError(e);
                            }
                            return super.indexTranslogOperations(operations, totalTranslogOps);
                        }
                    });
            pendingDocActiveWithExtraDocIndexed.await();
            assertThat(pendingDocDone.getCount(), equalTo(1L));
            {
                final long expectedDocs = docs + 2L;
                assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
                // recovery has not completed, therefore the global checkpoint can have advanced on the primary
                assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
                // the pending document is not done, the checkpoints can not have advanced on the replica
                assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 1));
                assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 1));
            }

            // wait for recovery to enter the translog phase
            phaseTwoStartLatch.countDown();

            // wait for the translog phase to complete and the recovery to block global checkpoint advancement
            awaitBusy(() -> shards.getPrimary().pendingInSync());
            {
                shards.index(new IndexRequest(index.getName(), "type", "last").source("{}", XContentType.JSON));
                final long expectedDocs = docs + 3L;
                assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
                // recovery is now in the process of being completed, therefore the global checkpoint can not have advanced on the primary
                assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 2));
                assertThat(replica.getLocalCheckpoint(), lessThan(expectedDocs - 2));
                assertThat(replica.getGlobalCheckpoint(), lessThan(expectedDocs - 2));
            }

            replicaEngineFactory.releaseLatchedIndexers();
            pendingDocDone.await();
            recoveryFuture.get();
            {
                final long expectedDocs = docs + 3L;
                assertBusy(() -> {
                    assertThat(shards.getPrimary().getLocalCheckpoint(), equalTo(expectedDocs - 1));
                    assertThat(shards.getPrimary().getGlobalCheckpoint(), equalTo(expectedDocs - 1));
                    assertThat(replica.getLocalCheckpoint(), equalTo(expectedDocs - 1));
                    // the global checkpoint advances can only advance here if a background global checkpoint sync fires
                    assertThat(replica.getGlobalCheckpoint(), anyOf(equalTo(expectedDocs - 1), equalTo(expectedDocs - 2)));
                });
            }
        }
    }

    public static class BlockingTarget extends RecoveryTarget {

        private final CountDownLatch recoveryBlocked;
        private final CountDownLatch releaseRecovery;
        private final RecoveryState.Stage stageToBlock;
        static final EnumSet<RecoveryState.Stage> SUPPORTED_STAGES =
            EnumSet.of(RecoveryState.Stage.INDEX, RecoveryState.Stage.TRANSLOG, RecoveryState.Stage.FINALIZE);
        private final Logger logger;

        public BlockingTarget(RecoveryState.Stage stageToBlock, CountDownLatch recoveryBlocked, CountDownLatch releaseRecovery,
                              IndexShard shard, DiscoveryNode sourceNode, PeerRecoveryTargetService.RecoveryListener listener,
                              Logger logger) {
            super(shard, sourceNode, listener, version -> {});
            this.recoveryBlocked = recoveryBlocked;
            this.releaseRecovery = releaseRecovery;
            this.stageToBlock = stageToBlock;
            this.logger = logger;
            if (SUPPORTED_STAGES.contains(stageToBlock) == false) {
                throw new UnsupportedOperationException(stageToBlock + " is not supported");
            }
        }

        private boolean hasBlocked() {
            return recoveryBlocked.getCount() == 0;
        }

        private void blockIfNeeded(RecoveryState.Stage currentStage) {
            if (currentStage == stageToBlock) {
                logger.info("--> blocking recovery on stage [{}]", currentStage);
                recoveryBlocked.countDown();
                try {
                    releaseRecovery.await();
                    logger.info("--> recovery continues from stage [{}]", currentStage);
                } catch (InterruptedException e) {
                    throw new RuntimeException("blockage released");
                }
            }
        }

        @Override
        public long indexTranslogOperations(List<Translog.Operation> operations, int totalTranslogOps) throws IOException {
            if (hasBlocked() == false) {
                blockIfNeeded(RecoveryState.Stage.TRANSLOG);
            }
            return super.indexTranslogOperations(operations, totalTranslogOps);
        }

        @Override
        public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaData) throws IOException {
            blockIfNeeded(RecoveryState.Stage.INDEX);
            super.cleanFiles(totalTranslogOps, sourceMetaData);
        }

        @Override
        public void finalizeRecovery(long globalCheckpoint) {
            if (hasBlocked() == false) {
                // it maybe that not ops have been transferred, block now
                blockIfNeeded(RecoveryState.Stage.TRANSLOG);
            }
            blockIfNeeded(RecoveryState.Stage.FINALIZE);
            super.finalizeRecovery(globalCheckpoint);
        }

    }

    static class BlockingEngineFactory implements EngineFactory, AutoCloseable {

        private final List<CountDownLatch> blocks = new ArrayList<>();

        private final AtomicReference<CountDownLatch> blockReference = new AtomicReference<>();
        private final AtomicReference<CountDownLatch> blockedIndexers = new AtomicReference<>();

        public synchronized void latchIndexers() {
            final CountDownLatch block = new CountDownLatch(1);
            blocks.add(block);
            blockedIndexers.set(new CountDownLatch(1));
            assert blockReference.compareAndSet(null, block);
        }

        public void awaitIndexersLatch() throws InterruptedException {
            blockedIndexers.get().await();
        }

        public synchronized void allowIndexing() {
            final CountDownLatch previous = blockReference.getAndSet(null);
            assert previous == null || blocks.contains(previous);
        }

        public synchronized void releaseLatchedIndexers() {
            allowIndexing();
            blocks.forEach(CountDownLatch::countDown);
            blocks.clear();
        }

        @Override
        public Engine newReadWriteEngine(final EngineConfig config) {
            return InternalEngineTests.createInternalEngine(
                    (directory, writerConfig) ->
                            new IndexWriter(directory, writerConfig) {
                                @Override
                                public long addDocument(final Iterable<? extends IndexableField> doc) throws IOException {
                                    final CountDownLatch block = blockReference.get();
                                    if (block != null) {
                                        final CountDownLatch latch = blockedIndexers.get();
                                        if (latch != null) {
                                            latch.countDown();
                                        }
                                        try {
                                            block.await();
                                        } catch (InterruptedException e) {
                                            throw new AssertionError(e);
                                        }
                                    }
                                    return super.addDocument(doc);
                                }
                            },
                    null,
                    config);
        }

        @Override
        public void close() throws Exception {
            releaseLatchedIndexers();
        }

    }

}