Skip to content

Commit 306e920

Browse files
[Remote Store] Add checks to skip remote uploads after shard is closed (#13998) (#14035)
(cherry picked from commit b01b6e8) Signed-off-by: Shourya Dutta Biswas <114977491+shourya035@users.noreply.github.com> Co-authored-by: github-actions[bot] <github-actions[bot]@users.noreply.github.com>
1 parent a1f7c82 commit 306e920

File tree

2 files changed

+93
-3
lines changed

2 files changed

+93
-3
lines changed

server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -170,6 +170,13 @@ protected boolean performAfterRefreshWithPermit(boolean didRefresh) {
170170
* @return true if sync is needed
171171
*/
172172
private boolean shouldSync(boolean didRefresh, boolean skipPrimaryTermCheck) {
173+
// Ignore syncing segments if the underlying shard is closed
174+
// This also makes sure that retries are not scheduled for shards
175+
// with failed syncSegments invocation after they are closed
176+
if (shardClosed()) {
177+
logger.info("Shard is already closed. Not attempting sync to remote store");
178+
return false;
179+
}
173180
boolean shouldSync = didRefresh // If the readers change, didRefresh is always true.
174181
// The third condition exists for uploading the zero state segments where the refresh has not changed the reader
175182
// reference, but it is important to upload the zero state segments so that the restore does not break.
@@ -608,6 +615,15 @@ public void onFailure(String file) {
608615
};
609616
}
610617

618+
/**
619+
* Checks if the underlying IndexShard instance is closed
620+
*
621+
* @return true if it is closed, false otherwise
622+
*/
623+
private boolean shardClosed() {
624+
return indexShard.state() == IndexShardState.CLOSED;
625+
}
626+
611627
@Override
612628
protected Logger getLogger() {
613629
return logger;

server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java

Lines changed: 77 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@
4747
import java.util.Map;
4848
import java.util.Objects;
4949
import java.util.concurrent.CountDownLatch;
50+
import java.util.concurrent.TimeUnit;
5051
import java.util.concurrent.atomic.AtomicLong;
5152

5253
import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE;
@@ -469,6 +470,25 @@ public void testRefreshPersistentFailure() throws Exception {
469470
assertFalse("remote store should not in sync", tuple.v1().isRemoteSegmentStoreInSync());
470471
}
471472

473+
public void testRefreshPersistentFailureAndIndexShardClosed() throws Exception {
474+
int succeedOnAttempt = 3;
475+
int closeShardOnAttempt = 1;
476+
CountDownLatch refreshCountLatch = new CountDownLatch(1);
477+
CountDownLatch successLatch = new CountDownLatch(10);
478+
Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> tuple = mockIndexShardWithRetryAndScheduleRefresh(
479+
succeedOnAttempt,
480+
refreshCountLatch,
481+
successLatch,
482+
true,
483+
closeShardOnAttempt
484+
);
485+
// Giving 10ms for some iterations of remote refresh upload
486+
Thread.sleep(TimeUnit.SECONDS.toMillis(2));
487+
RemoteStoreRefreshListener listener = tuple.v1();
488+
assertFalse("remote store should not in sync", listener.isRemoteSegmentStoreInSync());
489+
assertFalse(listener.getRetryScheduledStatus());
490+
}
491+
472492
private void assertNoLagAndTotalUploadsFailed(RemoteSegmentTransferTracker segmentTracker, long totalUploadsFailed) throws Exception {
473493
assertBusy(() -> {
474494
assertEquals(0, segmentTracker.getBytesLag());
@@ -547,6 +567,49 @@ private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIn
547567
return mockIndexShardWithRetryAndScheduleRefresh(succeedOnAttempt, refreshCountLatch, successLatch, 1, noOpLatch);
548568
}
549569

570+
private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIndexShardWithRetryAndScheduleRefresh(
571+
int totalAttempt,
572+
CountDownLatch refreshCountLatch,
573+
CountDownLatch successLatch,
574+
int checkpointPublishSucceedOnAttempt,
575+
CountDownLatch reachedCheckpointPublishLatch,
576+
boolean mockPrimaryTerm,
577+
boolean testUploadTimeout
578+
) throws IOException {
579+
return mockIndexShardWithRetryAndScheduleRefresh(
580+
totalAttempt,
581+
refreshCountLatch,
582+
successLatch,
583+
checkpointPublishSucceedOnAttempt,
584+
reachedCheckpointPublishLatch,
585+
mockPrimaryTerm,
586+
testUploadTimeout,
587+
false,
588+
0
589+
);
590+
}
591+
592+
private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIndexShardWithRetryAndScheduleRefresh(
593+
int succeedOnAttempt,
594+
CountDownLatch refreshCountLatch,
595+
CountDownLatch successLatch,
596+
boolean closedShard,
597+
int closeShardAfterAttempt
598+
) throws IOException {
599+
CountDownLatch noOpLatch = new CountDownLatch(0);
600+
return mockIndexShardWithRetryAndScheduleRefresh(
601+
succeedOnAttempt,
602+
refreshCountLatch,
603+
successLatch,
604+
1,
605+
noOpLatch,
606+
true,
607+
false,
608+
closedShard,
609+
closeShardAfterAttempt
610+
);
611+
}
612+
550613
private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIndexShardWithRetryAndScheduleRefresh(
551614
int succeedOnAttempt,
552615
CountDownLatch refreshCountLatch,
@@ -561,7 +624,9 @@ private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIn
561624
succeedCheckpointPublishOnAttempt,
562625
reachedCheckpointPublishLatch,
563626
true,
564-
false
627+
false,
628+
false,
629+
0
565630
);
566631
}
567632

@@ -572,7 +637,9 @@ private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIn
572637
int succeedCheckpointPublishOnAttempt,
573638
CountDownLatch reachedCheckpointPublishLatch,
574639
boolean mockPrimaryTerm,
575-
boolean testUploadTimeout
640+
boolean testUploadTimeout,
641+
boolean closeShard,
642+
int closeShardAfterAttempt
576643
) throws IOException {
577644
// Create index shard that we will be using to mock different methods in IndexShard for the unit test
578645
indexShard = newStartedShard(
@@ -600,7 +667,6 @@ private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIn
600667
IndexShard shard = mock(IndexShard.class);
601668
Store store = mock(Store.class);
602669
when(shard.store()).thenReturn(store);
603-
when(shard.state()).thenReturn(IndexShardState.STARTED);
604670
when(store.directory()).thenReturn(indexShard.store().directory());
605671

606672
// Mock (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory())
@@ -662,6 +728,14 @@ private Tuple<RemoteStoreRefreshListener, RemoteStoreStatsTrackerFactory> mockIn
662728
return indexShard.getLatestReplicationCheckpoint();
663729
})).when(shard).computeReplicationCheckpoint(any());
664730

731+
doAnswer((invocationOnMock -> {
732+
if (closeShard && counter.get() == closeShardAfterAttempt) {
733+
logger.info("Closing shard...");
734+
return IndexShardState.CLOSED;
735+
}
736+
return IndexShardState.STARTED;
737+
})).when(shard).state();
738+
665739
doAnswer(invocation -> {
666740
if (Objects.nonNull(successLatch)) {
667741
successLatch.countDown();

0 commit comments

Comments
 (0)