Skip to content

Commit

Permalink
Fix SegmentReplicationUsingRemoteStoreIT#testDropPrimaryDuringReplica…
Browse files Browse the repository at this point in the history
…tion.

This test is failing because a concurrent flush can wipe out an old commit file
while we are in the remote store refresh listener. The listener will fetch the latest infos from the reader which will reference a segments_n tht has been deleted by an incoming flush.

To fix this, InternalEngine.getSegmentInfosSnapshot will always hold the latest index commit.  If the commit gen is higher than that on the active reader, we will read and return the infos associated with the commit.

Signed-off-by: Marc Handalian <handalm@amazon.com>
  • Loading branch information
mch2 committed Aug 23, 2023
1 parent d1678ba commit 446b567
Show file tree
Hide file tree
Showing 9 changed files with 143 additions and 61 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -62,15 +62,4 @@ public void setup() {
public void teardown() {
assertAcked(clusterAdmin().prepareDeleteRepository(REPOSITORY_NAME));
}

@Override
public void testPressureServiceStats() throws Exception {
super.testPressureServiceStats();
}

@Override
@AwaitsFix(bugUrl = "https://github.com/opensearch-project/OpenSearch/issues/8059")
public void testDropPrimaryDuringReplication() throws Exception {
super.testDropPrimaryDuringReplication();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -2134,18 +2134,10 @@ protected SegmentInfos getLastCommittedSegmentInfos() {

@Override
protected SegmentInfos getLatestSegmentInfos() {
OpenSearchDirectoryReader reader = null;
try {
reader = internalReaderManager.acquire();
return ((StandardDirectoryReader) reader.getDelegate()).getSegmentInfos();
try (final GatedCloseable<SegmentInfos> snapshot = getSegmentInfosSnapshot()) {
return snapshot.get();
} catch (IOException e) {
throw new EngineException(shardId, e.getMessage(), e);
} finally {
try {
internalReaderManager.release(reader);
} catch (IOException e) {
throw new EngineException(shardId, e.getMessage(), e);
}
}
}

Expand All @@ -2156,15 +2148,26 @@ protected SegmentInfos getLatestSegmentInfos() {
*/
@Override
public GatedCloseable<SegmentInfos> getSegmentInfosSnapshot() {
final SegmentInfos segmentInfos = getLatestSegmentInfos();
final SegmentInfos segmentInfos;
final OpenSearchDirectoryReader reader;
final GatedCloseable<IndexCommit> lastIndexCommit = acquireLastIndexCommit(false);
try {
indexWriter.incRefDeleter(segmentInfos);
reader = internalReaderManager.acquire();
SegmentInfos readerInfos = ((StandardDirectoryReader) reader.getDelegate()).getSegmentInfos();
// if the latest index commit on disk has a higher gen than the reader load that instead.
// This ensures our commit file will exist throughout upload process.
if (lastIndexCommit.get().getGeneration() > readerInfos.getGeneration()) {
segmentInfos = SegmentInfos.readCommit(store.directory(), lastIndexCommit.get().getSegmentsFileName());
} else {
segmentInfos = readerInfos;
}
} catch (IOException e) {
throw new EngineException(shardId, e.getMessage(), e);
}
return new GatedCloseable<>(segmentInfos, () -> {
try {
indexWriter.decRefDeleter(segmentInfos);
internalReaderManager.release(reader);
lastIndexCommit.close();
} catch (AlreadyClosedException e) {
logger.warn("Engine is already closed.", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.opensearch.action.LatchedActionListener;
import org.opensearch.action.bulk.BackoffPolicy;
import org.opensearch.action.support.GroupedActionListener;
import org.opensearch.common.collect.Tuple;
import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.common.logging.Loggers;
import org.opensearch.common.unit.TimeValue;
Expand Down Expand Up @@ -196,7 +197,6 @@ private boolean syncSegments() {
);
return true;
}
ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint();
beforeSegmentsSync();
long refreshTimeMs = segmentTracker.getLocalRefreshTimeMs(), refreshClockTimeMs = segmentTracker.getLocalRefreshClockTimeMs();
long refreshSeqNo = segmentTracker.getLocalRefreshSeqNo();
Expand All @@ -211,13 +211,14 @@ private boolean syncSegments() {
if (isRefreshAfterCommit()) {
remoteDirectory.deleteStaleSegmentsAsync(LAST_N_METADATA_FILES_TO_KEEP);
}

try (GatedCloseable<SegmentInfos> segmentInfosGatedCloseable = indexShard.getSegmentInfosSnapshot()) {
final Tuple<GatedCloseable<SegmentInfos>, ReplicationCheckpoint> tuple = indexShard.getLatestSegmentInfosAndCheckpoint();
try (GatedCloseable<SegmentInfos> segmentInfosGatedCloseable = tuple.v1()) {
final ReplicationCheckpoint checkpoint = tuple.v2();
SegmentInfos segmentInfos = segmentInfosGatedCloseable.get();
assert segmentInfos.getGeneration() == checkpoint.getSegmentsGen() : "SegmentInfos generation: "
+ segmentInfos.getGeneration()
+ " does not match metadata generation: "
+ checkpoint.getSegmentsGen();
assert segmentInfos.getVersion() == checkpoint.getSegmentInfosVersion() : "SegmentInfos version: "
+ segmentInfos.getVersion()

Check warning on line 219 in server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java

View check run for this annotation

Codecov / codecov/patch

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

Added line #L219 was not covered by tests
+ " does not match metadata version: "
+ checkpoint.getSegmentInfosVersion();

Check warning on line 221 in server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java

View check run for this annotation

Codecov / codecov/patch

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

Added line #L221 was not covered by tests
// Capture replication checkpoint before uploading the segments as upload can take some time and checkpoint can
// move.
long lastRefreshedCheckpoint = ((InternalEngine) indexShard.getEngine()).lastRefreshedCheckpoint();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@

package org.opensearch.indices.replication.common;

import org.apache.lucene.index.IndexCommit;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.ByteBuffersDataOutput;
import org.apache.lucene.store.ByteBuffersIndexOutput;
Expand Down Expand Up @@ -38,7 +37,6 @@ public class CopyState extends AbstractRefCounted {
private final ReplicationCheckpoint replicationCheckpoint;
private final Map<String, StoreFileMetadata> metadataMap;
private final byte[] infosBytes;
private GatedCloseable<IndexCommit> commitRef;
private final IndexShard shard;

public CopyState(ReplicationCheckpoint requestedReplicationCheckpoint, IndexShard shard) throws IOException {
Expand All @@ -51,7 +49,6 @@ public CopyState(ReplicationCheckpoint requestedReplicationCheckpoint, IndexShar
this.replicationCheckpoint = latestSegmentInfosAndCheckpoint.v2();
SegmentInfos segmentInfos = this.segmentInfosRef.get();
this.metadataMap = shard.store().getSegmentMetadataMap(segmentInfos);
this.commitRef = shard.acquireLastIndexCommit(false);

ByteBuffersDataOutput buffer = new ByteBuffersDataOutput();
// resource description and name are not used, but resource description cannot be null
Expand All @@ -65,10 +62,6 @@ public CopyState(ReplicationCheckpoint requestedReplicationCheckpoint, IndexShar
protected void closeInternal() {
try {
segmentInfosRef.close();
// commitRef may be null if there were no pending delete files
if (commitRef != null) {
commitRef.close();
}
} catch (IOException e) {
throw new UncheckedIOException(e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -156,6 +156,7 @@
import org.opensearch.threadpool.ThreadPool;
import org.hamcrest.MatcherAssert;
import org.hamcrest.Matchers;
import org.junit.Assert;

import java.io.Closeable;
import java.io.IOException;
Expand All @@ -165,6 +166,7 @@
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
Expand Down Expand Up @@ -7530,16 +7532,86 @@ public void testMaxDocsOnReplica() throws Exception {
}
}

public void testGetSegmentInfosSnapshot() throws IOException {
public void testGetSegmentInfosSnapshot_AllSnapshotFilesPreservedAcrossCommit() throws Exception {
IOUtils.close(store, engine);
Store store = createStore();
InternalEngine engine = spy(createEngine(store, createTempDir()));
GatedCloseable<SegmentInfos> segmentInfosSnapshot = engine.getSegmentInfosSnapshot();
assertNotNull(segmentInfosSnapshot);
assertNotNull(segmentInfosSnapshot.get());
verify(engine, times(1)).getLatestSegmentInfos();
store.close();
engine.close();
store = createStore();
engine = createEngine(store, createTempDir());
List<Engine.Operation> operations = generateHistoryOnReplica(
randomIntBetween(1, 100),
randomBoolean(),
randomBoolean(),
randomBoolean()
);
for (Engine.Operation op : operations) {
applyOperation(engine, op);
}
engine.refresh("test");
try (GatedCloseable<SegmentInfos> snapshot = engine.getSegmentInfosSnapshot()) {
Collection<String> files = snapshot.get().files(true);
Set<String> localFiles = Set.of(store.directory().listAll());
for (String file : files) {
assertTrue("Local directory contains file " + file, localFiles.contains(file));
}

engine.flush(true, true);

try (
final GatedCloseable<SegmentInfos> snapshotAfterFlush = engine.getSegmentInfosSnapshot();
final GatedCloseable<IndexCommit> commit = engine.acquireLastIndexCommit(false)
) {
final SegmentInfos segmentInfos = snapshotAfterFlush.get();
assertNotEquals(segmentInfos.getSegmentsFileName(), snapshot.get().getSegmentsFileName());
assertEquals(commit.get().getSegmentsFileName(), segmentInfos.getSegmentsFileName());
}

// original files are preserved.
localFiles = Set.of(store.directory().listAll());
for (String file : files) {
assertTrue("Local directory contains file " + file, localFiles.contains(file));
}
}
}

public void testGetSegmentInfosSnapshot_LatestCommitOnDiskHasHigherGenThanReader() throws Exception {
IOUtils.close(store, engine);
store = createStore();
engine = createEngine(store, createTempDir());
// to simulate this we need concurrent flush/refresh.
AtomicBoolean run = new AtomicBoolean(true);
AtomicInteger docId = new AtomicInteger(0);
Thread refresher = new Thread(() -> {
while (run.get()) {
try {
engine.index(indexForDoc(createParsedDoc(Integer.toString(docId.getAndIncrement()), null)));
engine.refresh("test");
getSnapshotAndAssertFilesExistLocally();
} catch (Exception e) {
Assert.fail();
}
}
});
refresher.start();
try {
for (int i = 0; i < 10; i++) {
engine.flush(true, true);
getSnapshotAndAssertFilesExistLocally();
}
} catch (Exception e) {
Assert.fail();
} finally {
run.set(false);
refresher.join();
}
}

private void getSnapshotAndAssertFilesExistLocally() throws IOException {
try (GatedCloseable<SegmentInfos> snapshot = engine.getSegmentInfosSnapshot()) {
Collection<String> files = snapshot.get().files(true);
Set<String> localFiles = Set.of(store.directory().listAll());
for (String file : files) {
assertTrue("Local directory contains file " + file, localFiles.contains(file));
}
}
}

public void testGetProcessedLocalCheckpoint() throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.util.Version;
import org.opensearch.cluster.metadata.IndexMetadata;
import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.common.settings.Settings;
import org.opensearch.common.util.FeatureFlags;
import org.opensearch.index.engine.DocIdSeqNoAndSource;
Expand Down Expand Up @@ -204,11 +205,14 @@ public void testReplicaCommitsInfosBytesOnRecovery() throws Exception {
Set.of("segments_3"),
primary.remoteStore().readLastCommittedSegmentsInfo().files(true)
);
MatcherAssert.assertThat(
"Segments are referenced in memory only",
primaryEngine.getSegmentInfosSnapshot().get().files(false),
containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs")
);

try (final GatedCloseable<SegmentInfos> segmentInfosSnapshot = primaryEngine.getSegmentInfosSnapshot()) {
MatcherAssert.assertThat(
"Segments are referenced in memory only",
segmentInfosSnapshot.get().files(false),
containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs")
);
}

final IndexShard replica = shards.addReplica(remotePath);
replica.store().createEmpty(Version.LATEST);
Expand Down Expand Up @@ -238,11 +242,15 @@ public void testReplicaCommitsInfosBytesOnRecovery() throws Exception {
latestReplicaCommit.files(true),
containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs", "segments_5")
);
MatcherAssert.assertThat(
"Segments are referenced in memory",
replicaEngine.getSegmentInfosSnapshot().get().files(false),
containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs")
);

try (final GatedCloseable<SegmentInfos> segmentInfosSnapshot = replicaEngine.getSegmentInfosSnapshot()) {
MatcherAssert.assertThat(
"Segments are referenced in memory",
segmentInfosSnapshot.get().files(false),
containsInAnyOrder("_0.cfe", "_0.si", "_0.cfs")
);
}

final Store.RecoveryDiff recoveryDiff = Store.segmentReplicationDiff(
primary.getSegmentMetadataMap(),
replica.getSegmentMetadataMap()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -508,6 +508,13 @@ private Tuple<RemoteStoreRefreshListener, RemoteStorePressureService> mockIndexS
return indexShard.getSegmentInfosSnapshot();
}).when(shard).getSegmentInfosSnapshot();

doAnswer((invocation -> {
if (counter.incrementAndGet() <= succeedOnAttempt) {
throw new RuntimeException("Inducing failure in upload");
}
return indexShard.getLatestSegmentInfosAndCheckpoint();
})).when(shard).getLatestSegmentInfosAndCheckpoint();

doAnswer(invocation -> {
if (Objects.nonNull(successLatch)) {
successLatch.countDown();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@
import org.junit.Assert;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand Down Expand Up @@ -786,17 +787,24 @@ protected void assertDocCounts(IndexShard indexShard, int expectedPersistedDocCo
}

protected void resolveCheckpointInfoResponseListener(ActionListener<CheckpointInfoResponse> listener, IndexShard primary) {
final CopyState copyState;
try {
final CopyState copyState = new CopyState(
copyState = new CopyState(
ReplicationCheckpoint.empty(primary.shardId, primary.getLatestReplicationCheckpoint().getCodec()),
primary
);
listener.onResponse(
new CheckpointInfoResponse(copyState.getCheckpoint(), copyState.getMetadataMap(), copyState.getInfosBytes())
);
} catch (IOException e) {
logger.error("Unexpected error computing CopyState", e);
Assert.fail("Failed to compute copyState");
throw new UncheckedIOException(e);
}

try {
listener.onResponse(
new CheckpointInfoResponse(copyState.getCheckpoint(), copyState.getMetadataMap(), copyState.getInfosBytes())
);
} finally {
copyState.decRef();
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -180,6 +180,7 @@ public void onFailure(Exception e) {
assertEquals(e.getClass(), OpenSearchException.class);
}
});
copyState.decRef();
}

public void testReplicationAlreadyRunning() throws IOException {
Expand Down

0 comments on commit 446b567

Please sign in to comment.