From c04190edc358ec40c69efdbfda536859a1657814 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 18 Jul 2023 12:18:06 -0700 Subject: [PATCH 01/26] [Segment Replication] Refactor remote replication source Signed-off-by: Suraj Singh --- .../java/org/opensearch/core/index/Index.java | 12 +++ .../opensearch/core/index/shard/ShardId.java | 13 +++ .../opensearch/index/shard/IndexShard.java | 97 +++++++++++++------ .../shard/RemoteStoreRefreshListener.java | 8 +- .../opensearch/index/shard/StoreRecovery.java | 2 +- .../store/RemoteSegmentStoreDirectory.java | 10 +- .../org/opensearch/index/store/Store.java | 9 +- .../metadata/RemoteSegmentMetadata.java | 35 ++++--- .../recovery/PeerRecoveryTargetService.java | 2 +- .../replication/GetSegmentFilesResponse.java | 4 + .../RemoteStoreReplicationSource.java | 13 +-- .../replication/SegmentReplicationTarget.java | 40 ++++---- .../checkpoint/ReplicationCheckpoint.java | 20 ++++ ...tReplicationWithRemoteIndexShardTests.java | 2 +- .../RemoteSegmentStoreDirectoryTests.java | 6 +- .../RemoteSegmentMetadataHandlerTests.java | 3 +- .../RemoteStoreReplicationSourceTests.java | 2 +- 17 files changed, 186 insertions(+), 92 deletions(-) diff --git a/libs/core/src/main/java/org/opensearch/core/index/Index.java b/libs/core/src/main/java/org/opensearch/core/index/Index.java index fc5c5152a5500..50c3ccd2d5382 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/Index.java +++ b/libs/core/src/main/java/org/opensearch/core/index/Index.java @@ -32,6 +32,8 @@ package org.opensearch.core.index; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; import org.opensearch.core.ParseField; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -134,6 +136,16 @@ public static Index fromXContent(final XContentParser parser) throws IOException return INDEX_PARSER.parse(parser, null).build(); } + public void writeTo(IndexOutput out) throws IOException { + out.writeString(name); + out.writeString(uuid); + } + + public Index(IndexInput in) throws IOException { + this.name = in.readString(); + this.uuid = in.readString(); + } + /** * Builder for Index objects. Used by ObjectParser instances only. * diff --git a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java index b01121c3f30d4..9fb8c1d66b649 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java +++ b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java @@ -32,6 +32,8 @@ package org.opensearch.core.index.shard; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -147,4 +149,15 @@ public int compareTo(ShardId o) { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { return builder.value(toString()); } + + public void writeTo(IndexOutput out) throws IOException { + index.writeTo(out); + out.writeVInt(shardId); + } + + public ShardId(IndexInput in) throws IOException { + index = new Index(in); + shardId = in.readVInt(); + hashCode = computeHashCode(); + } } diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index bb5088866edb6..a0b3cdbacd866 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -2326,7 +2326,7 @@ private void innerOpenEngineAndTranslog(LongSupplier globalCheckpointSupplier, b assert currentEngineReference.get() == null : "engine is running"; verifyNotClosed(); if (indexSettings.isRemoteStoreEnabled() && syncFromRemote) { - syncSegmentsFromRemoteSegmentStore(false, true, true); + syncSegmentsFromRemoteSegmentStore(false, true); } if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { if (syncFromRemote) { @@ -4555,7 +4555,7 @@ public void close() throws IOException { }; IOUtils.close(currentEngineReference.getAndSet(readOnlyEngine)); if (indexSettings.isRemoteStoreEnabled()) { - syncSegmentsFromRemoteSegmentStore(false, true, true); + syncSegmentsFromRemoteSegmentStore(false, true); } if (indexSettings.isRemoteTranslogStoreEnabled() && shardRouting.primary()) { syncRemoteTranslogAndUpdateGlobalCheckpoint(); @@ -4612,14 +4612,53 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { RemoteFsTranslog.download(repository, shardId, getThreadPool(), shardPath().resolveTranslog(), logger); } + /** + * Segment Replication method + * + * Downloads specified segments from remote store + * @param filesToFetch Files to download from remote store + * + */ + public List syncSegmentsFromRemoteSegmentStore(List filesToFetch) throws IOException { + assert indexSettings.isSegRepEnabled() && indexSettings.isRemoteStoreEnabled(); + logger.trace("Downloading segments files from remote store {}", filesToFetch); + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = getRemoteDirectory(); + RemoteSegmentMetadata remoteSegmentMetadata = remoteSegmentStoreDirectory.init(); + List downloadedSegments = new ArrayList<>(); + if (remoteSegmentMetadata != null) { + try { + store.incRef(); + remoteStore.incRef(); + final Directory storeDirectory = store.directory(); + logger.info("--> storeDirectory {}", storeDirectory.getClass()); + String segmentNFile = null; + for (StoreFileMetadata fileMetadata : filesToFetch) { + String file = fileMetadata.name(); + logger.info("--> Copying file {}", file); + storeDirectory.copyFrom(remoteSegmentStoreDirectory, file, file, IOContext.DEFAULT); + downloadedSegments.add(fileMetadata); + if (file.startsWith(IndexFileNames.SEGMENTS)) { + assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; + segmentNFile = file; + } + } + storeDirectory.sync(downloadedSegments.stream().map(metadata -> metadata.name()).collect(Collectors.toList())); + } finally { + store.decRef(); + remoteStore.decRef(); + logger.trace("Downloaded segments from remote store {}", downloadedSegments); + } + } + return downloadedSegments; + } + /** * Downloads segments from remote segment store. * @param overrideLocal flag to override local segment files with those in remote store * @param refreshLevelSegmentSync last refresh checkpoint is used if true, commit checkpoint otherwise - * @param shouldCommit if the shard requires committing the changes after sync from remote. * @throws IOException if exception occurs while reading segments from remote store */ - public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean refreshLevelSegmentSync, boolean shouldCommit) + public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean refreshLevelSegmentSync) throws IOException { assert indexSettings.isRemoteStoreEnabled(); logger.info("Downloading segments from remote segment store"); @@ -4662,35 +4701,31 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re remoteSegmentMetadata.getGeneration() ); // Replicas never need a local commit - if (shouldCommit) { - if (this.shardRouting.primary()) { - long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); - // Following code block makes sure to use SegmentInfosSnapshot in the remote store if generation differs - // with local filesystem. If local filesystem already has segments_N+2 and infosSnapshot has generation N, - // after commit, there would be 2 files that would be created segments_N+1 and segments_N+2. With the - // policy of preserving only the latest commit, we will delete segments_N+1 which in fact is the part of the - // latest commit. - Optional localMaxSegmentInfos = localSegmentFiles.stream() - .filter(file -> file.startsWith(IndexFileNames.SEGMENTS)) - .max(Comparator.comparingLong(SegmentInfos::generationFromSegmentsFileName)); - if (localMaxSegmentInfos.isPresent() - && infosSnapshot.getGeneration() < SegmentInfos.generationFromSegmentsFileName(localMaxSegmentInfos.get()) - - 1) { - // If remote translog is not enabled, local translog will be created with different UUID. - // This fails in Store.trimUnsafeCommits() as translog UUID of checkpoint and SegmentInfos needs - // to be same. Following code block make sure to have the same UUID. - if (indexSettings.isRemoteTranslogStoreEnabled() == false) { - SegmentInfos localSegmentInfos = store.readLastCommittedSegmentsInfo(); - Map userData = new HashMap<>(infosSnapshot.getUserData()); - userData.put(TRANSLOG_UUID_KEY, localSegmentInfos.userData.get(TRANSLOG_UUID_KEY)); - infosSnapshot.setUserData(userData, false); - } - storeDirectory.deleteFile(localMaxSegmentInfos.get()); + if (this.shardRouting.primary()) { + long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); + // Following code block makes sure to use SegmentInfosSnapshot in the remote store if generation differs + // with local filesystem. If local filesystem already has segments_N+2 and infosSnapshot has generation N, + // after commit, there would be 2 files that would be created segments_N+1 and segments_N+2. With the + // policy of preserving only the latest commit, we will delete segments_N+1 which in fact is the part of the + // latest commit. + Optional localMaxSegmentInfos = localSegmentFiles.stream() + .filter(file -> file.startsWith(IndexFileNames.SEGMENTS)) + .max(Comparator.comparingLong(SegmentInfos::generationFromSegmentsFileName)); + if (localMaxSegmentInfos.isPresent() + && infosSnapshot.getGeneration() < SegmentInfos.generationFromSegmentsFileName(localMaxSegmentInfos.get()) + - 1) { + // If remote translog is not enabled, local translog will be created with different UUID. + // This fails in Store.trimUnsafeCommits() as translog UUID of checkpoint and SegmentInfos needs + // to be same. Following code block make sure to have the same UUID. + if (indexSettings.isRemoteTranslogStoreEnabled() == false) { + SegmentInfos localSegmentInfos = store.readLastCommittedSegmentsInfo(); + Map userData = new HashMap<>(infosSnapshot.getUserData()); + userData.put(TRANSLOG_UUID_KEY, localSegmentInfos.userData.get(TRANSLOG_UUID_KEY)); + infosSnapshot.setUserData(userData, false); } - store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + storeDirectory.deleteFile(localMaxSegmentInfos.get()); } - } else { - finalizeReplication(infosSnapshot); + store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); } } } diff --git a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java index 4a70ff04770d3..49da1231fd4fc 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -220,7 +220,7 @@ private synchronized boolean syncSegments() { public void onResponse(Void unused) { try { // Start metadata file upload - uploadMetadata(localSegmentsPostRefresh, segmentInfos); + uploadMetadata(localSegmentsPostRefresh, segmentInfos, checkpoint); clearStaleFilesFromLocalSegmentChecksumMap(localSegmentsPostRefresh); onSuccessfulSegmentsSync( refreshTimeMs, @@ -327,7 +327,7 @@ private boolean isRefreshAfterCommit() throws IOException { && !remoteDirectory.containsFile(lastCommittedLocalSegmentFileName, getChecksumOfLocalFile(lastCommittedLocalSegmentFileName))); } - void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos segmentInfos) throws IOException { + void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos segmentInfos, ReplicationCheckpoint replicationCheckpoint) throws IOException { final long maxSeqNo = ((InternalEngine) indexShard.getEngine()).currentOngoingRefreshCheckpoint(); SegmentInfos segmentInfosSnapshot = segmentInfos.clone(); Map userData = segmentInfosSnapshot.getUserData(); @@ -344,8 +344,8 @@ void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos se localSegmentsPostRefresh, segmentInfosSnapshot, storeDirectory, - indexShard.getOperationPrimaryTerm(), - translogFileGeneration + translogFileGeneration, + replicationCheckpoint ); } } diff --git a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java index 5897fa7d513d7..2c8a186a6ed53 100644 --- a/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java +++ b/server/src/main/java/org/opensearch/index/shard/StoreRecovery.java @@ -530,7 +530,7 @@ private void recoverFromRemoteStore(IndexShard indexShard) throws IndexShardReco remoteStore.incRef(); try { // Download segments from remote segment store - indexShard.syncSegmentsFromRemoteSegmentStore(true, true, true); + indexShard.syncSegmentsFromRemoteSegmentStore(true, true); if (store.directory().listAll().length == 0) { store.createEmpty(indexShard.indexSettings().getIndexVersionCreated().luceneVersion); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 8ee267cb67e68..5cdd512451a91 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -44,6 +44,7 @@ import org.opensearch.index.store.lockmanager.RemoteStoreLockManager; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.threadpool.ThreadPool; import java.io.FileNotFoundException; @@ -610,12 +611,12 @@ public void uploadMetadata( Collection segmentFiles, SegmentInfos segmentInfosSnapshot, Directory storeDirectory, - long primaryTerm, - long translogGeneration + long translogGeneration, + ReplicationCheckpoint replicationCheckpoint ) throws IOException { synchronized (this) { String metadataFilename = MetadataFilenameUtils.getMetadataFilename( - primaryTerm, + replicationCheckpoint.getPrimaryTerm(), segmentInfosSnapshot.getGeneration(), translogGeneration, metadataUploadCounter.incrementAndGet(), @@ -646,8 +647,7 @@ public void uploadMetadata( new RemoteSegmentMetadata( RemoteSegmentMetadata.fromMapOfStrings(uploadedSegments), segmentInfoSnapshotByteArray, - primaryTerm, - segmentInfosSnapshot.getGeneration() + replicationCheckpoint ) ); } diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index a67b87f58110c..427b0b57faaaa 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -845,22 +845,23 @@ private void cleanupFiles(Collection filesToConsiderForCleanup, String r * @param tmpToFileName Map of temporary replication file to actual file name * @param infosBytes bytes[] of SegmentInfos supposed to be sent over by primary excluding segment_N file * @param segmentsGen segment generation number - * @param consumer consumer for generated SegmentInfos + * @param finalizeConsumer consumer for generated SegmentInfos * @throws IOException Exception while reading store and building segment infos */ public void buildInfosFromBytes( Map tmpToFileName, byte[] infosBytes, long segmentsGen, - CheckedConsumer consumer + CheckedConsumer finalizeConsumer, + CheckedConsumer, IOException> renameConsumer ) throws IOException { metadataLock.writeLock().lock(); try { final List values = new ArrayList<>(tmpToFileName.values()); incRefFileDeleter(values); try { - renameTempFilesSafe(tmpToFileName); - consumer.accept(buildSegmentInfos(infosBytes, segmentsGen)); + renameConsumer.accept(tmpToFileName); + finalizeConsumer.accept(buildSegmentInfos(infosBytes, segmentsGen)); } finally { decRefFileDeleter(values); } diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 9a479346ff711..a94429eb2ecc6 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -15,6 +15,7 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.opensearch.index.store.RemoteSegmentStoreDirectory; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; /** * Metadata object for Remote Segment @@ -38,19 +39,29 @@ public class RemoteSegmentMetadata { private final byte[] segmentInfosBytes; - private final long primaryTerm; - private final long generation; +// private final long primaryTerm; +// private final long generation; +// +// private final long version; +// +// private final long length; +// +// private final String codec; + + public ReplicationCheckpoint getReplicationCheckpoint() { + return replicationCheckpoint; + } + + private final ReplicationCheckpoint replicationCheckpoint; public RemoteSegmentMetadata( Map metadata, byte[] segmentInfosBytes, - long primaryTerm, - long generation + ReplicationCheckpoint replicationCheckpoint ) { this.metadata = metadata; this.segmentInfosBytes = segmentInfosBytes; - this.generation = generation; - this.primaryTerm = primaryTerm; + this.replicationCheckpoint = replicationCheckpoint; } /** @@ -66,11 +77,11 @@ public byte[] getSegmentInfosBytes() { } public long getGeneration() { - return generation; + return replicationCheckpoint.getSegmentsGen(); } public long getPrimaryTerm() { - return primaryTerm; + return replicationCheckpoint.getPrimaryTerm(); } /** @@ -99,19 +110,17 @@ public static Map f public void write(IndexOutput out) throws IOException { out.writeMapOfStrings(toMapOfStrings()); - out.writeLong(generation); - out.writeLong(primaryTerm); + replicationCheckpoint.writeTo(out); out.writeLong(segmentInfosBytes.length); out.writeBytes(segmentInfosBytes, segmentInfosBytes.length); } public static RemoteSegmentMetadata read(IndexInput indexInput) throws IOException { Map metadata = indexInput.readMapOfStrings(); - long generation = indexInput.readLong(); - long primaryTerm = indexInput.readLong(); + ReplicationCheckpoint replicationCheckpoint = new ReplicationCheckpoint(indexInput); int byteArraySize = (int) indexInput.readLong(); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); - return new RemoteSegmentMetadata(RemoteSegmentMetadata.fromMapOfStrings(metadata), segmentInfosBytes, primaryTerm, generation); + return new RemoteSegmentMetadata(RemoteSegmentMetadata.fromMapOfStrings(metadata), segmentInfosBytes, replicationCheckpoint); } } diff --git a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java index 0ba57a9ee7f65..386b2e0e8192d 100644 --- a/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/opensearch/indices/recovery/PeerRecoveryTargetService.java @@ -245,7 +245,7 @@ private void doRecovery(final long recoveryId, final StartRecoveryRequest preExi indexShard.prepareForIndexRecovery(); final boolean hasRemoteSegmentStore = indexShard.indexSettings().isRemoteStoreEnabled(); if (hasRemoteSegmentStore) { - indexShard.syncSegmentsFromRemoteSegmentStore(false, false, true); + indexShard.syncSegmentsFromRemoteSegmentStore(false, false); } final boolean hasRemoteTranslog = recoveryTarget.state().getPrimary() == false && indexShard.isRemoteTranslogEnabled(); final boolean hasNoTranslog = indexShard.indexSettings().isRemoteSnapshot(); diff --git a/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java b/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java index 89d50a17464a6..222b1cfaa8574 100644 --- a/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java +++ b/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java @@ -23,6 +23,10 @@ */ public class GetSegmentFilesResponse extends TransportResponse { + public List getFiles() { + return files; + } + List files; public GetSegmentFilesResponse(List files) { diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index c5be7635782af..8e71fd2295e35 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -33,7 +33,7 @@ */ public class RemoteStoreReplicationSource implements SegmentReplicationSource { - private static final Logger logger = LogManager.getLogger(PrimaryShardReplicationSource.class); + private static final Logger logger = LogManager.getLogger(RemoteStoreReplicationSource.class); private final IndexShard indexShard; @@ -77,8 +77,8 @@ public void getCheckpointMetadata( ) ) ); - // TODO: GET current checkpoint from remote store. - listener.onResponse(new CheckpointInfoResponse(checkpoint, metadataMap, null)); + logger.info("--> Sending empty checkpoint"); + listener.onResponse(new CheckpointInfoResponse(mdFile.getReplicationCheckpoint(), metadataMap, mdFile.getSegmentInfosBytes())); } catch (Exception e) { listener.onFailure(e); } @@ -93,8 +93,9 @@ public void getSegmentFiles( ActionListener listener ) { try { - indexShard.syncSegmentsFromRemoteSegmentStore(false, true, false); - listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); + List downloadedFiles = indexShard.syncSegmentsFromRemoteSegmentStore(filesToFetch); + assert downloadedFiles.size() == filesToFetch.size() && downloadedFiles.containsAll(filesToFetch); + listener.onResponse(new GetSegmentFilesResponse(downloadedFiles)); } catch (Exception e) { listener.onFailure(e); } @@ -102,6 +103,6 @@ public void getSegmentFiles( @Override public String getDescription() { - return "remote store"; + return "RemoteStoreReplicationSource"; } } diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index 2e0f5a8c0ad1f..cac2509882f16 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -38,8 +38,11 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; /** * Represents the target of a replication event. @@ -148,7 +151,7 @@ public void startReplication(ActionListener listener) { final StepListener checkpointInfoListener = new StepListener<>(); final StepListener getFilesListener = new StepListener<>(); - logger.trace(new ParameterizedMessage("Starting Replication Target: {}", description())); + logger.info(new ParameterizedMessage("Starting Replication Target: {}", description())); // Get list of files to copy from this checkpoint. state.setStage(SegmentReplicationState.Stage.GET_CHECKPOINT_INFO); cancellableThreads.checkForCancel(); @@ -158,11 +161,13 @@ public void startReplication(ActionListener listener) { final List filesToFetch = getFiles(checkpointInfo); state.setStage(SegmentReplicationState.Stage.GET_FILES); cancellableThreads.checkForCancel(); + logger.info("--> Before getFiles {}", Arrays.toString(indexShard.store().directory().listAll())); source.getSegmentFiles(getId(), checkpointInfo.getCheckpoint(), filesToFetch, indexShard, getFilesListener); }, listener::onFailure); getFilesListener.whenComplete(response -> { - finalizeReplication(checkpointInfoListener.result()); + logger.info("--> After getFiles {}", Arrays.toString(indexShard.store().directory().listAll())); + finalizeReplication(checkpointInfoListener.result(), getFilesListener.result()); listener.onResponse(null); }, listener::onFailure); } @@ -193,23 +198,26 @@ private List getFiles(CheckpointInfoResponse checkpointInfo) return diff.missing; } - private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws OpenSearchCorruptionException { - // TODO: Refactor the logic so that finalize doesn't have to be invoked for remote store as source - if (source instanceof RemoteStoreReplicationSource) { - state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); - return; - } + private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, GetSegmentFilesResponse getSegmentFilesResponse) throws OpenSearchCorruptionException { cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); Store store = null; try { store = store(); store.incRef(); + Map tempFileNames = null; + if (this.indexShard.indexSettings().isRemoteStoreEnabled() == true) { + tempFileNames = getSegmentFilesResponse.getFiles() != null ? getSegmentFilesResponse.getFiles().stream().collect(Collectors.toMap(StoreFileMetadata::name, StoreFileMetadata::name)) : Collections.emptyMap(); + } else { + tempFileNames = multiFileWriter.getTempFileNames(); + } + logger.info("--> tempFileNames {} checkpointInfoResponse.getCheckpoint().getSegmentsGen() {}", tempFileNames, checkpointInfoResponse.getCheckpoint().getSegmentsGen()); store.buildInfosFromBytes( - multiFileWriter.getTempFileNames(), + tempFileNames, checkpointInfoResponse.getInfosBytes(), checkpointInfoResponse.getCheckpoint().getSegmentsGen(), - indexShard::finalizeReplication + indexShard::finalizeReplication, + this.indexShard.indexSettings().isRemoteStoreEnabled() == true ? (files) -> {}: (files) -> indexShard.store().renameTempFilesSafe(files) ); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. @@ -247,23 +255,13 @@ private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) } } - /** - * This method formats our byte[] containing the primary's SegmentInfos into lucene's {@link ChecksumIndexInput} that can be - * passed to SegmentInfos.readCommit - */ - private ChecksumIndexInput toIndexInput(byte[] input) { - return new BufferedChecksumIndexInput( - new ByteBuffersIndexInput(new ByteBuffersDataInput(Arrays.asList(ByteBuffer.wrap(input))), "SegmentInfos") - ); - } - /** * Trigger a cancellation, this method will not close the target a subsequent call to #fail is required from target service. */ @Override public void cancel(String reason) { if (finished.get() == false) { - logger.trace(new ParameterizedMessage("Cancelling replication for target {}", description())); + logger.info(new ParameterizedMessage("Cancelling replication for target {}", description())); cancellableThreads.cancel(reason); source.cancel(); } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index 7549f3450e7f2..e002c35a430ec 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -8,6 +8,8 @@ package org.opensearch.indices.replication.checkpoint; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; import org.opensearch.Version; import org.opensearch.common.Nullable; import org.opensearch.core.common.io.stream.StreamInput; @@ -186,4 +188,22 @@ public String toString() { + codec + '}'; } + + public void writeTo(IndexOutput out) throws IOException { + shardId.writeTo(out); + out.writeLong(primaryTerm); + out.writeLong(segmentsGen); + out.writeLong(segmentInfosVersion); + out.writeLong(length); + out.writeString(codec); + } + + public ReplicationCheckpoint(IndexInput in) throws IOException { + shardId = new ShardId(in); + primaryTerm = in.readLong(); + segmentsGen = in.readLong(); + segmentInfosVersion = in.readLong(); + length = in.readLong(); + codec = in.readString(); + } } diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java index a67b60d6128d1..d522e21261b37 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java @@ -36,7 +36,7 @@ public void testReplicaSyncingFromRemoteStore() throws IOException { assertDocs(primaryShard, "1", "2"); flushShard(primaryShard); - replicaShard.syncSegmentsFromRemoteSegmentStore(true, true, false); + replicaShard.syncSegmentsFromRemoteSegmentStore(true, true); assertDocs(replicaShard, "1", "2"); closeShards(primaryShard, replicaShard); } diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 7c765cf5df0be..dcb5cc33b1f59 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -8,6 +8,7 @@ package org.opensearch.index.store; +import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; @@ -43,6 +44,7 @@ import org.opensearch.index.store.lockmanager.RemoteStoreMetadataLockManager; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -681,7 +683,7 @@ public void testUploadMetadataEmpty() throws IOException { Collection segmentFiles = List.of("_s1.si", "_s1.cfe", "_s3.cfs"); assertThrows( NoSuchFileException.class, - () -> remoteSegmentStoreDirectory.uploadMetadata(segmentFiles, segmentInfos, storeDirectory, 12L, 34L) + () -> remoteSegmentStoreDirectory.uploadMetadata(segmentFiles, segmentInfos, storeDirectory, 12L, ReplicationCheckpoint.empty(indexShard.shardId(), Codec.getDefault().getName())) ); } @@ -717,7 +719,7 @@ public void testUploadMetadataNonEmpty() throws IOException { when(storeDirectory.createOutput(startsWith("metadata__" + primaryTermLong + "__" + generationLong), eq(IOContext.DEFAULT))) .thenReturn(indexOutput); - remoteSegmentStoreDirectory.uploadMetadata(segInfos.files(true), segInfos, storeDirectory, primaryTerm, generation); + remoteSegmentStoreDirectory.uploadMetadata(segInfos.files(true), segInfos, storeDirectory, generation, indexShard.getLatestReplicationCheckpoint()); verify(remoteMetadataDirectory).copyFrom( eq(storeDirectory), diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java index 2fee77ab563c0..c891623619bde 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java @@ -106,8 +106,7 @@ public void testWriteContent() throws IOException { RemoteSegmentMetadata remoteSegmentMetadata = new RemoteSegmentMetadata( RemoteSegmentMetadata.fromMapOfStrings(expectedOutput), segmentInfosBytes, - 1234, - 1234 + indexShard.getLatestReplicationCheckpoint() ); remoteSegmentMetadataHandler.writeContent(indexOutput, remoteSegmentMetadata); indexOutput.close(); diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index 04b5aa58ea485..6b6c90a2b2254 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -190,7 +190,7 @@ public void testGetSegmentFilesFailure() throws IOException { ); Mockito.doThrow(new RuntimeException("testing")) .when(mockShard) - .syncSegmentsFromRemoteSegmentStore(Mockito.anyBoolean(), Mockito.anyBoolean(), Mockito.anyBoolean()); + .syncSegmentsFromRemoteSegmentStore(Mockito.anyBoolean(), Mockito.anyBoolean()); assertThrows(ExecutionException.class, () -> { final PlainActionFuture res = PlainActionFuture.newFuture(); replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), mockShard, res); From 2d57ddc24eac6802f8cbcc236bcb93a02fe9476c Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 18 Jul 2023 17:16:16 -0700 Subject: [PATCH 02/26] Unit test updates Signed-off-by: Suraj Singh --- .../opensearch/index/shard/IndexShard.java | 4 +-- .../shard/RemoteStoreRefreshListener.java | 3 +- .../store/RemoteSegmentStoreDirectory.java | 3 +- .../org/opensearch/index/store/Store.java | 2 +- .../metadata/RemoteSegmentMetadata.java | 16 ++++----- .../RemoteStoreReplicationSource.java | 1 - .../replication/SegmentReplicationTarget.java | 19 +++++----- .../RemoteSegmentStoreDirectoryTests.java | 35 ++++++++++++------- .../RemoteStoreReplicationSourceTests.java | 28 +++------------ 9 files changed, 51 insertions(+), 60 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index a0b3cdbacd866..fd5fc49a113b3 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4630,7 +4630,6 @@ public List syncSegmentsFromRemoteSegmentStore(List storeDirectory {}", storeDirectory.getClass()); String segmentNFile = null; for (StoreFileMetadata fileMetadata : filesToFetch) { String file = fileMetadata.name(); @@ -4658,8 +4657,7 @@ public List syncSegmentsFromRemoteSegmentStore(List localSegmentsPostRefresh, SegmentInfos segmentInfos, ReplicationCheckpoint replicationCheckpoint) throws IOException { + void uploadMetadata(Collection localSegmentsPostRefresh, SegmentInfos segmentInfos, ReplicationCheckpoint replicationCheckpoint) + throws IOException { final long maxSeqNo = ((InternalEngine) indexShard.getEngine()).currentOngoingRefreshCheckpoint(); SegmentInfos segmentInfosSnapshot = segmentInfos.clone(); Map userData = segmentInfosSnapshot.getUserData(); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 5cdd512451a91..8dfdb3e2c8e06 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -604,7 +604,8 @@ public boolean containsFile(String localFilename, String checksum) { * @param segmentFiles segment files that are part of the shard at the time of the latest refresh * @param segmentInfosSnapshot SegmentInfos bytes to store as part of metadata file * @param storeDirectory instance of local directory to temporarily create metadata file before upload - * @param primaryTerm primary term to be used in the name of metadata file + * @param translogGeneration translog generation + * @param replicationCheckpoint ReplicationCheckpoint of primary shard * @throws IOException in case of I/O error while uploading the metadata file */ public void uploadMetadata( diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index 427b0b57faaaa..04bfe49ba4e70 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -853,7 +853,7 @@ public void buildInfosFromBytes( byte[] infosBytes, long segmentsGen, CheckedConsumer finalizeConsumer, - CheckedConsumer, IOException> renameConsumer + CheckedConsumer, IOException> renameConsumer ) throws IOException { metadataLock.writeLock().lock(); try { diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index a94429eb2ecc6..01f1e5ce20879 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -39,14 +39,14 @@ public class RemoteSegmentMetadata { private final byte[] segmentInfosBytes; -// private final long primaryTerm; -// private final long generation; -// -// private final long version; -// -// private final long length; -// -// private final String codec; + // private final long primaryTerm; + // private final long generation; + // + // private final long version; + // + // private final long length; + // + // private final String codec; public ReplicationCheckpoint getReplicationCheckpoint() { return replicationCheckpoint; diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 8e71fd2295e35..36b58de318959 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -77,7 +77,6 @@ public void getCheckpointMetadata( ) ) ); - logger.info("--> Sending empty checkpoint"); listener.onResponse(new CheckpointInfoResponse(mdFile.getReplicationCheckpoint(), metadataMap, mdFile.getSegmentInfosBytes())); } catch (Exception e) { listener.onFailure(e); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index cac2509882f16..8cd8d782d1004 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -13,10 +13,6 @@ import org.apache.lucene.index.IndexFormatTooNewException; import org.apache.lucene.index.IndexFormatTooOldException; import org.apache.lucene.store.AlreadyClosedException; -import org.apache.lucene.store.BufferedChecksumIndexInput; -import org.apache.lucene.store.ByteBuffersDataInput; -import org.apache.lucene.store.ByteBuffersIndexInput; -import org.apache.lucene.store.ChecksumIndexInput; import org.opensearch.OpenSearchCorruptionException; import org.opensearch.OpenSearchException; import org.opensearch.action.ActionListener; @@ -36,7 +32,6 @@ import org.opensearch.indices.replication.common.ReplicationTarget; import java.io.IOException; -import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -198,7 +193,8 @@ private List getFiles(CheckpointInfoResponse checkpointInfo) return diff.missing; } - private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, GetSegmentFilesResponse getSegmentFilesResponse) throws OpenSearchCorruptionException { + private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, GetSegmentFilesResponse getSegmentFilesResponse) + throws OpenSearchCorruptionException { cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); Store store = null; @@ -207,17 +203,22 @@ private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, store.incRef(); Map tempFileNames = null; if (this.indexShard.indexSettings().isRemoteStoreEnabled() == true) { - tempFileNames = getSegmentFilesResponse.getFiles() != null ? getSegmentFilesResponse.getFiles().stream().collect(Collectors.toMap(StoreFileMetadata::name, StoreFileMetadata::name)) : Collections.emptyMap(); + tempFileNames = getSegmentFilesResponse.getFiles() != null + ? getSegmentFilesResponse.getFiles() + .stream() + .collect(Collectors.toMap(StoreFileMetadata::name, StoreFileMetadata::name)) + : Collections.emptyMap(); } else { tempFileNames = multiFileWriter.getTempFileNames(); } - logger.info("--> tempFileNames {} checkpointInfoResponse.getCheckpoint().getSegmentsGen() {}", tempFileNames, checkpointInfoResponse.getCheckpoint().getSegmentsGen()); store.buildInfosFromBytes( tempFileNames, checkpointInfoResponse.getInfosBytes(), checkpointInfoResponse.getCheckpoint().getSegmentsGen(), indexShard::finalizeReplication, - this.indexShard.indexSettings().isRemoteStoreEnabled() == true ? (files) -> {}: (files) -> indexShard.store().renameTempFilesSafe(files) + this.indexShard.indexSettings().isRemoteStoreEnabled() == true + ? (files) -> {} + : (files) -> indexShard.store().renameTempFilesSafe(files) ); } catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) { // this is a fatal exception at this stage. diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index dcb5cc33b1f59..82ab2d4b79b81 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -8,7 +8,6 @@ package org.opensearch.index.store; -import org.apache.lucene.codecs.Codec; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; @@ -45,6 +44,7 @@ import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadataHandler; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.threadpool.ThreadPool; import java.io.IOException; @@ -57,8 +57,6 @@ import java.util.Set; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; -import java.util.HashMap; -import java.util.Collection; import java.util.concurrent.ExecutorService; import static org.mockito.Mockito.mock; @@ -103,7 +101,10 @@ public void setup() throws IOException { ); testUploadTracker = new TestUploadListener(); - Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT).build(); + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .build(); ExecutorService executorService = OpenSearchExecutors.newDirectExecutorService(); indexShard = newStartedShard(false, indexSettings, new NRTReplicationEngineFactory()); @@ -262,7 +263,7 @@ private Map getDummyMetadata(String prefix, int commitGeneration * @return ByteArrayIndexInput: metadata file bytes with header and footer * @throws IOException IOException */ - private ByteArrayIndexInput createMetadataFileBytes(Map segmentFilesMap, long generation, long primaryTerm) + private ByteArrayIndexInput createMetadataFileBytes(Map segmentFilesMap, ReplicationCheckpoint replicationCheckpoint) throws IOException { ByteBuffersDataOutput byteBuffersIndexOutput = new ByteBuffersDataOutput(); segmentInfos.write(new ByteBuffersIndexOutput(byteBuffersIndexOutput, "", "")); @@ -272,8 +273,7 @@ private ByteArrayIndexInput createMetadataFileBytes(Map segmentF OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("segment metadata", "metadata output stream", output, 4096); CodecUtil.writeHeader(indexOutput, RemoteSegmentMetadata.METADATA_CODEC, RemoteSegmentMetadata.CURRENT_VERSION); indexOutput.writeMapOfStrings(segmentFilesMap); - indexOutput.writeLong(generation); - indexOutput.writeLong(primaryTerm); + replicationCheckpoint.writeTo(indexOutput); indexOutput.writeLong(byteArray.length); indexOutput.writeBytes(byteArray, byteArray.length); CodecUtil.writeFooter(indexOutput); @@ -311,13 +311,13 @@ private Map> populateMetadata() throws IOException { ); when(remoteMetadataDirectory.openInput(metadataFilename, IOContext.DEFAULT)).thenAnswer( - I -> createMetadataFileBytes(metadataFilenameContentMapping.get(metadataFilename), 23, 12) + I -> createMetadataFileBytes(metadataFilenameContentMapping.get(metadataFilename), indexShard.getLatestReplicationCheckpoint()) ); when(remoteMetadataDirectory.openInput(metadataFilename2, IOContext.DEFAULT)).thenAnswer( - I -> createMetadataFileBytes(metadataFilenameContentMapping.get(metadataFilename2), 13, 12) + I -> createMetadataFileBytes(metadataFilenameContentMapping.get(metadataFilename2), indexShard.getLatestReplicationCheckpoint()) ); when(remoteMetadataDirectory.openInput(metadataFilename3, IOContext.DEFAULT)).thenAnswer( - I -> createMetadataFileBytes(metadataFilenameContentMapping.get(metadataFilename3), 38, 10) + I -> createMetadataFileBytes(metadataFilenameContentMapping.get(metadataFilename3), indexShard.getLatestReplicationCheckpoint()) ); return metadataFilenameContentMapping; @@ -653,7 +653,9 @@ public void testContainsFile() throws IOException { metadata.put("_0.cfe", "_0.cfe::_0.cfe__" + UUIDs.base64UUID() + "::1234::512::" + Version.LATEST.major); metadata.put("_0.cfs", "_0.cfs::_0.cfs__" + UUIDs.base64UUID() + "::2345::1024::" + Version.LATEST.major); - when(remoteMetadataDirectory.openInput(metadataFilename, IOContext.DEFAULT)).thenReturn(createMetadataFileBytes(metadata, 1, 5)); + when(remoteMetadataDirectory.openInput(metadataFilename, IOContext.DEFAULT)).thenReturn( + createMetadataFileBytes(metadata, indexShard.getLatestReplicationCheckpoint()) + ); remoteSegmentStoreDirectory.init(); @@ -678,12 +680,19 @@ public void testContainsFile() throws IOException { public void testUploadMetadataEmpty() throws IOException { Directory storeDirectory = mock(Directory.class); IndexOutput indexOutput = mock(IndexOutput.class); - when(storeDirectory.createOutput(startsWith("metadata__12__o"), eq(IOContext.DEFAULT))).thenReturn(indexOutput); + final long primaryTerm = indexShard.getOperationPrimaryTerm(); + when(storeDirectory.createOutput(startsWith("metadata__" + primaryTerm + "__o"), eq(IOContext.DEFAULT))).thenReturn(indexOutput); Collection segmentFiles = List.of("_s1.si", "_s1.cfe", "_s3.cfs"); assertThrows( NoSuchFileException.class, - () -> remoteSegmentStoreDirectory.uploadMetadata(segmentFiles, segmentInfos, storeDirectory, 12L, ReplicationCheckpoint.empty(indexShard.shardId(), Codec.getDefault().getName())) + () -> remoteSegmentStoreDirectory.uploadMetadata( + segmentFiles, + segmentInfos, + storeDirectory, + 34L, + indexShard.getLatestReplicationCheckpoint() + ) ); } diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index 6b6c90a2b2254..cca706283dfbb 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -87,13 +87,7 @@ public void tearDown() throws Exception { public void testGetCheckpointMetadata() throws ExecutionException, InterruptedException { when(mockShard.getSegmentInfosSnapshot()).thenReturn(indexShard.getSegmentInfosSnapshot()); - final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( - indexShard.shardId(), - PRIMARY_TERM, - SEGMENTS_GEN, - VERSION, - Codec.getDefault().getName() - ); + final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); final PlainActionFuture res = PlainActionFuture.newFuture(); replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); @@ -103,13 +97,7 @@ public void testGetCheckpointMetadata() throws ExecutionException, InterruptedEx } public void testGetCheckpointMetadataFailure() { - final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( - indexShard.shardId(), - PRIMARY_TERM, - SEGMENTS_GEN, - VERSION, - Codec.getDefault().getName() - ); + final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); when(mockShard.getSegmentInfosSnapshot()).thenThrow(new RuntimeException("test")); @@ -176,21 +164,15 @@ public void testGetSegmentFiles() throws ExecutionException, InterruptedExceptio replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), indexShard, res); GetSegmentFilesResponse response = res.get(); assert (response.files.isEmpty()); - assertEquals("remote store", replicationSource.getDescription()); + assertEquals("RemoteStoreReplicationSource", replicationSource.getDescription()); } public void testGetSegmentFilesFailure() throws IOException { - final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( - indexShard.shardId(), - PRIMARY_TERM, - SEGMENTS_GEN, - VERSION, - Codec.getDefault().getName() - ); + final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); Mockito.doThrow(new RuntimeException("testing")) .when(mockShard) - .syncSegmentsFromRemoteSegmentStore(Mockito.anyBoolean(), Mockito.anyBoolean()); + .syncSegmentsFromRemoteSegmentStore(Mockito.any()); assertThrows(ExecutionException.class, () -> { final PlainActionFuture res = PlainActionFuture.newFuture(); replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), mockShard, res); From 929d824f0aa809895b72a4494961aae16100c3a0 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 18 Jul 2023 17:32:41 -0700 Subject: [PATCH 03/26] Self review Signed-off-by: Suraj Singh --- .../opensearch/index/shard/IndexShard.java | 39 ------------------- .../RemoteStoreReplicationSource.java | 37 ++++++++++++++++-- .../RemoteStoreReplicationSourceTests.java | 4 +- 3 files changed, 35 insertions(+), 45 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index fd5fc49a113b3..fae0129bb5a50 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4612,45 +4612,6 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { RemoteFsTranslog.download(repository, shardId, getThreadPool(), shardPath().resolveTranslog(), logger); } - /** - * Segment Replication method - * - * Downloads specified segments from remote store - * @param filesToFetch Files to download from remote store - * - */ - public List syncSegmentsFromRemoteSegmentStore(List filesToFetch) throws IOException { - assert indexSettings.isSegRepEnabled() && indexSettings.isRemoteStoreEnabled(); - logger.trace("Downloading segments files from remote store {}", filesToFetch); - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = getRemoteDirectory(); - RemoteSegmentMetadata remoteSegmentMetadata = remoteSegmentStoreDirectory.init(); - List downloadedSegments = new ArrayList<>(); - if (remoteSegmentMetadata != null) { - try { - store.incRef(); - remoteStore.incRef(); - final Directory storeDirectory = store.directory(); - String segmentNFile = null; - for (StoreFileMetadata fileMetadata : filesToFetch) { - String file = fileMetadata.name(); - logger.info("--> Copying file {}", file); - storeDirectory.copyFrom(remoteSegmentStoreDirectory, file, file, IOContext.DEFAULT); - downloadedSegments.add(fileMetadata); - if (file.startsWith(IndexFileNames.SEGMENTS)) { - assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; - segmentNFile = file; - } - } - storeDirectory.sync(downloadedSegments.stream().map(metadata -> metadata.name()).collect(Collectors.toList())); - } finally { - store.decRef(); - remoteStore.decRef(); - logger.trace("Downloaded segments from remote store {}", downloadedSegments); - } - } - return downloadedSegments; - } - /** * Downloads segments from remote segment store. * @param overrideLocal flag to override local segment files with those in remote store diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 36b58de318959..a97cec2c904e2 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -10,7 +10,10 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.index.IndexFileNames; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; +import org.apache.lucene.store.IOContext; import org.apache.lucene.util.Version; import org.opensearch.action.ActionListener; import org.opensearch.index.shard.IndexShard; @@ -21,6 +24,7 @@ import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -92,9 +96,36 @@ public void getSegmentFiles( ActionListener listener ) { try { - List downloadedFiles = indexShard.syncSegmentsFromRemoteSegmentStore(filesToFetch); - assert downloadedFiles.size() == filesToFetch.size() && downloadedFiles.containsAll(filesToFetch); - listener.onResponse(new GetSegmentFilesResponse(downloadedFiles)); + logger.trace("Downloading segments files from remote store {}", filesToFetch); + FilterDirectory remoteStoreDirectory = (FilterDirectory) indexShard.remoteStore().directory(); + FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) byteSizeCachingStoreDirectory + .getDelegate(); + RemoteSegmentMetadata remoteSegmentMetadata = remoteSegmentStoreDirectory.init(); + List downloadedSegments = new ArrayList<>(); + if (remoteSegmentMetadata != null) { + try { + indexShard.store().incRef(); + indexShard.remoteStore().incRef(); + final Directory storeDirectory = indexShard.store().directory(); + String segmentNFile = null; + for (StoreFileMetadata fileMetadata : filesToFetch) { + String file = fileMetadata.name(); + storeDirectory.copyFrom(remoteSegmentStoreDirectory, file, file, IOContext.DEFAULT); + downloadedSegments.add(fileMetadata); + if (file.startsWith(IndexFileNames.SEGMENTS)) { + assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; + segmentNFile = file; + } + } + storeDirectory.sync(downloadedSegments.stream().map(metadata -> metadata.name()).collect(Collectors.toList())); + } finally { + indexShard.store().decRef(); + indexShard.remoteStore().decRef(); + logger.trace("Downloaded segments from remote store {}", downloadedSegments); + } + } + listener.onResponse(new GetSegmentFilesResponse(downloadedSegments)); } catch (Exception e) { listener.onFailure(e); } diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index cca706283dfbb..208ea596bdb67 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -170,9 +170,7 @@ public void testGetSegmentFiles() throws ExecutionException, InterruptedExceptio public void testGetSegmentFilesFailure() throws IOException { final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); - Mockito.doThrow(new RuntimeException("testing")) - .when(mockShard) - .syncSegmentsFromRemoteSegmentStore(Mockito.any()); + Mockito.doThrow(new RuntimeException("testing")).when(mockShard).store(); assertThrows(ExecutionException.class, () -> { final PlainActionFuture res = PlainActionFuture.newFuture(); replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), mockShard, res); From eb297465fd3e9c80458180bc97a9bdf9bea2ebbe Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 18 Jul 2023 17:39:03 -0700 Subject: [PATCH 04/26] Self review Signed-off-by: Suraj Singh --- .../store/remote/metadata/RemoteSegmentMetadata.java | 9 --------- .../indices/replication/GetSegmentFilesResponse.java | 8 ++++---- .../indices/replication/SegmentReplicationTarget.java | 7 ++----- 3 files changed, 6 insertions(+), 18 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 01f1e5ce20879..befec685754c2 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -39,15 +39,6 @@ public class RemoteSegmentMetadata { private final byte[] segmentInfosBytes; - // private final long primaryTerm; - // private final long generation; - // - // private final long version; - // - // private final long length; - // - // private final String codec; - public ReplicationCheckpoint getReplicationCheckpoint() { return replicationCheckpoint; } diff --git a/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java b/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java index 222b1cfaa8574..33a84833f2418 100644 --- a/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java +++ b/server/src/main/java/org/opensearch/indices/replication/GetSegmentFilesResponse.java @@ -23,10 +23,6 @@ */ public class GetSegmentFilesResponse extends TransportResponse { - public List getFiles() { - return files; - } - List files; public GetSegmentFilesResponse(List files) { @@ -37,6 +33,10 @@ public GetSegmentFilesResponse(StreamInput out) throws IOException { out.readList(StoreFileMetadata::new); } + public List getFiles() { + return files; + } + @Override public void writeTo(StreamOutput out) throws IOException { out.writeCollection(files); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index 8cd8d782d1004..ee4490e94fdb5 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -32,7 +32,6 @@ import org.opensearch.indices.replication.common.ReplicationTarget; import java.io.IOException; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Locale; @@ -146,7 +145,7 @@ public void startReplication(ActionListener listener) { final StepListener checkpointInfoListener = new StepListener<>(); final StepListener getFilesListener = new StepListener<>(); - logger.info(new ParameterizedMessage("Starting Replication Target: {}", description())); + logger.trace(new ParameterizedMessage("Starting Replication Target: {}", description())); // Get list of files to copy from this checkpoint. state.setStage(SegmentReplicationState.Stage.GET_CHECKPOINT_INFO); cancellableThreads.checkForCancel(); @@ -156,12 +155,10 @@ public void startReplication(ActionListener listener) { final List filesToFetch = getFiles(checkpointInfo); state.setStage(SegmentReplicationState.Stage.GET_FILES); cancellableThreads.checkForCancel(); - logger.info("--> Before getFiles {}", Arrays.toString(indexShard.store().directory().listAll())); source.getSegmentFiles(getId(), checkpointInfo.getCheckpoint(), filesToFetch, indexShard, getFilesListener); }, listener::onFailure); getFilesListener.whenComplete(response -> { - logger.info("--> After getFiles {}", Arrays.toString(indexShard.store().directory().listAll())); finalizeReplication(checkpointInfoListener.result(), getFilesListener.result()); listener.onResponse(null); }, listener::onFailure); @@ -262,7 +259,7 @@ private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, @Override public void cancel(String reason) { if (finished.get() == false) { - logger.info(new ParameterizedMessage("Cancelling replication for target {}", description())); + logger.trace(new ParameterizedMessage("Cancelling replication for target {}", description())); cancellableThreads.cancel(reason); source.cancel(); } From 60126e5395c3bc5758e8440161bdaa387ebc9bd4 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Thu, 20 Jul 2023 15:24:02 -0700 Subject: [PATCH 05/26] Segregate shard level tests for node to node and remote store segment replication Signed-off-by: Suraj Singh --- .../replication/SegmentReplicationBaseIT.java | 20 - .../opensearch/index/shard/IndexShard.java | 4 +- .../store/RemoteSegmentStoreDirectory.java | 2 +- .../SegmentReplicationIndexShardTests.java | 612 +++--------------- ...licationWithNodeToNodeIndexShardTests.java | 469 ++++++++++++++ ...tReplicationWithRemoteIndexShardTests.java | 170 ++++- ...enSearchIndexLevelReplicationTestCase.java | 7 +- .../index/shard/IndexShardTestCase.java | 11 +- 8 files changed, 725 insertions(+), 570 deletions(-) create mode 100644 server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java index 64c6ebbb33482..cfb2e11c8c429 100644 --- a/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/indices/replication/SegmentReplicationBaseIT.java @@ -8,7 +8,6 @@ package org.opensearch.indices.replication; -import org.opensearch.action.admin.indices.replication.SegmentReplicationStatsResponse; import org.opensearch.action.search.SearchResponse; import org.opensearch.cluster.ClusterState; import org.opensearch.cluster.metadata.IndexMetadata; @@ -24,7 +23,6 @@ import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexService; -import org.opensearch.index.SegmentReplicationPerGroupStats; import org.opensearch.index.SegmentReplicationShardStats; import org.opensearch.index.engine.Engine; import org.opensearch.index.shard.IndexShard; @@ -134,24 +132,6 @@ protected void waitForSearchableDocs(long docCount, String... nodes) throws Exce waitForSearchableDocs(docCount, Arrays.stream(nodes).collect(Collectors.toList())); } - protected void waitForSegmentReplication(String node) throws Exception { - assertBusy(() -> { - SegmentReplicationStatsResponse segmentReplicationStatsResponse = client(node).admin() - .indices() - .prepareSegmentReplicationStats(INDEX_NAME) - .setDetailed(true) - .execute() - .actionGet(); - final SegmentReplicationPerGroupStats perGroupStats = segmentReplicationStatsResponse.getReplicationStats() - .get(INDEX_NAME) - .get(0); - assertEquals( - perGroupStats.getReplicaStats().stream().findFirst().get().getCurrentReplicationState().getStage(), - SegmentReplicationState.Stage.DONE - ); - }, 1, TimeUnit.MINUTES); - } - protected void verifyStoreContent() throws Exception { assertBusy(() -> { final ClusterState clusterState = getClusterState(); diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index fae0129bb5a50..f1d12ef383fc7 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4620,7 +4620,7 @@ public void syncTranslogFilesFromRemoteTranslog() throws IOException { */ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean refreshLevelSegmentSync) throws IOException { assert indexSettings.isRemoteStoreEnabled(); - logger.info("Downloading segments from remote segment store"); + logger.trace("Downloading segments from remote segment store"); RemoteSegmentStoreDirectory remoteDirectory = getRemoteDirectory(); // We need to call RemoteSegmentStoreDirectory.init() in order to get latest metadata of the files that // are uploaded to the remote segment store. @@ -4710,7 +4710,7 @@ public void syncSegmentsFromGivenRemoteSegmentStore( long primaryTerm, long commitGeneration ) throws IOException { - logger.info("Downloading segments from given remote segment store"); + logger.trace("Downloading segments from given remote segment store"); RemoteSegmentStoreDirectory remoteDirectory = null; if (remoteStore != null) { remoteDirectory = getRemoteDirectory(); diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 8dfdb3e2c8e06..2b5d3ff822757 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -201,7 +201,7 @@ public RemoteSegmentMetadata readLatestMetadataFile() throws IOException { return remoteSegmentMetadata; } - private RemoteSegmentMetadata readMetadataFile(String metadataFilename) throws IOException { + private RemoteSegmentMetadata readMetadataFile(String metadataFilename) throws IOException {S try (IndexInput indexInput = remoteMetadataDirectory.openInput(metadataFilename, IOContext.DEFAULT)) { byte[] metadataBytes = new byte[(int) indexInput.length()]; indexInput.readBytes(metadataBytes, 0, (int) indexInput.length()); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 9107606326150..1a33473f04768 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -33,7 +33,6 @@ import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.Engine; -import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; @@ -67,7 +66,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -75,14 +73,9 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import static java.util.Arrays.asList; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.mockito.Mockito.any; @@ -100,6 +93,67 @@ public class SegmentReplicationIndexShardTests extends OpenSearchIndexLevelRepli .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .build(); + protected SegmentReplicationTargetService segmentReplicationTargetService() { + RecoverySettings recoverySettings = new RecoverySettings( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + TransportService transportService = mock(TransportService.class); + IndicesService indicesService = mock(IndicesService.class); + SegmentReplicationSourceFactory sourceFactory = new SegmentReplicationSourceFactory( + transportService, + recoverySettings, + clusterService + ); + return new SegmentReplicationTargetService( + threadPool, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + mock(TransportService.class), + sourceFactory, + indicesService, + clusterService + ); + } + + protected ReplicationGroup getReplicationGroup(int numberOfReplicas) throws IOException { + return createGroup(numberOfReplicas, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory()); + } + + protected ReplicationGroup getReplicationGroup(int numberOfReplicas, String indexMapping) throws IOException { + return createGroup(numberOfReplicas, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory()); + } + + protected Settings getIndexSettings() { + return settings; + } + + protected IndexShard getStartedShard() throws IOException { + return newStartedShard(randomBoolean(), settings, new NRTReplicationEngineFactory()); + } + + /** + * Validates happy path of segment replication where primary index docs which are replicated to replica shards. Assertions + * made on doc count on both primary and replica. + * @throws Exception + */ + public void testReplication() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory());) { + shards.startAll(); + final IndexShard primaryShard = shards.getPrimary(); + final IndexShard replicaShard = shards.getReplicas().get(0); + + // index and replicate segments to replica. + int numDocs = randomIntBetween(10, 100); + shards.indexDocs(numDocs); + primaryShard.refresh("test"); + flushShard(primaryShard); + replicateSegments(primaryShard, List.of(replicaShard)); + + // Assertions + shards.assertAllEqual(numDocs); + } + } + /** * Test that latestReplicationCheckpoint returns null only for docrep enabled indices */ @@ -114,7 +168,7 @@ public void testReplicationCheckpointNullForDocRep() throws IOException { * Test that latestReplicationCheckpoint returns ReplicationCheckpoint for segrep enabled indices */ public void testReplicationCheckpointNotNullForSegRep() throws IOException { - final IndexShard indexShard = newStartedShard(randomBoolean(), settings, new NRTReplicationEngineFactory()); + final IndexShard indexShard = newStartedShard(randomBoolean(), getIndexSettings(), new NRTReplicationEngineFactory()); final ReplicationCheckpoint replicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); assertNotNull(replicationCheckpoint); closeShards(indexShard); @@ -127,7 +181,7 @@ public void testNRTReplicasDoNotAcceptRefreshListeners() throws IOException { } public void testSegmentInfosAndReplicationCheckpointTuple() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory())) { shards.startAll(); final IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); @@ -180,7 +234,7 @@ private void assertReplicationCheckpoint(IndexShard shard, SegmentInfos segmentI } public void testIsSegmentReplicationAllowed_WrongEngineType() throws IOException { - final IndexShard indexShard = newShard(false, settings, new InternalEngineFactory()); + final IndexShard indexShard = newShard(false, getIndexSettings(), new InternalEngineFactory()); assertFalse(indexShard.isSegmentReplicationAllowed()); closeShards(indexShard); } @@ -193,7 +247,7 @@ public void testIsSegmentReplicationAllowed_WrongEngineType() throws IOException */ public void testSegmentReplication_With_ReaderClosedConcurrently() throws Exception { String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; - try (ReplicationGroup shards = createGroup(1, settings, mappings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), mappings, new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primaryShard = shards.getPrimary(); final IndexShard replicaShard = shards.getReplicas().get(0); @@ -242,7 +296,7 @@ public void testSegmentReplication_With_ReaderClosedConcurrently() throws Except */ public void testSegmentReplication_With_EngineClosedConcurrently() throws Exception { String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; - try (ReplicationGroup shards = createGroup(1, settings, mappings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), mappings, new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primaryShard = shards.getPrimary(); final IndexShard replicaShard = shards.getReplicas().get(0); @@ -284,87 +338,9 @@ public void testSegmentReplication_With_EngineClosedConcurrently() throws Except } } - /** - * Verifies that commits on replica engine resulting from engine or reader close does not cleanup the temporary - * replication files from ongoing round of segment replication - */ - public void testTemporaryFilesNotCleanup() throws Exception { - String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; - try (ReplicationGroup shards = createGroup(1, settings, mappings, new NRTReplicationEngineFactory())) { - shards.startAll(); - IndexShard primaryShard = shards.getPrimary(); - final IndexShard replica = shards.getReplicas().get(0); - - // Step 1. Ingest numDocs documents, commit to create commit point on primary & replicate - final int numDocs = randomIntBetween(100, 200); - logger.info("--> Inserting documents {}", numDocs); - for (int i = 0; i < numDocs; i++) { - shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); - } - assertEqualTranslogOperations(shards, primaryShard); - primaryShard.flush(new FlushRequest().waitIfOngoing(true).force(true)); - replicateSegments(primaryShard, shards.getReplicas()); - shards.assertAllEqual(numDocs); - - // Step 2. Ingest numDocs documents again to create a new commit on primary - logger.info("--> Ingest {} docs again", numDocs); - for (int i = 0; i < numDocs; i++) { - shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); - } - assertEqualTranslogOperations(shards, primaryShard); - primaryShard.flush(new FlushRequest().waitIfOngoing(true).force(true)); - - // Step 3. Copy segment files to replica shard but prevent commit - final CountDownLatch countDownLatch = new CountDownLatch(1); - Map primaryMetadata; - try (final GatedCloseable segmentInfosSnapshot = primaryShard.getSegmentInfosSnapshot()) { - final SegmentInfos primarySegmentInfos = segmentInfosSnapshot.get(); - primaryMetadata = primaryShard.store().getSegmentMetadataMap(primarySegmentInfos); - } - final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); - final IndicesService indicesService = mock(IndicesService.class); - when(indicesService.getShardOrNull(replica.shardId)).thenReturn(replica); - final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( - threadPool, - new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), - mock(TransportService.class), - sourceFactory, - indicesService, - clusterService - ); - final Consumer runnablePostGetFiles = (indexShard) -> { - try { - Collection temporaryFiles = Stream.of(indexShard.store().directory().listAll()) - .filter(name -> name.startsWith(SegmentReplicationTarget.REPLICATION_PREFIX)) - .collect(Collectors.toList()); - - // Step 4. Perform a commit on replica shard. - NRTReplicationEngine engine = (NRTReplicationEngine) indexShard.getEngine(); - engine.updateSegments(engine.getSegmentInfosSnapshot().get()); - - // Step 5. Validate temporary files are not deleted from store. - Collection replicaStoreFiles = List.of(indexShard.store().directory().listAll()); - assertTrue(replicaStoreFiles.containsAll(temporaryFiles)); - } catch (IOException e) { - throw new RuntimeException(e); - } - }; - SegmentReplicationSource segmentReplicationSource = getSegmentReplicationSource( - primaryShard, - (repId) -> targetService.get(repId), - runnablePostGetFiles - ); - when(sourceFactory.get(any())).thenReturn(segmentReplicationSource); - targetService.startReplication(replica, getTargetListener(primaryShard, replica, primaryMetadata, countDownLatch)); - countDownLatch.await(30, TimeUnit.SECONDS); - assertEquals("Replication failed", 0, countDownLatch.getCount()); - shards.assertAllEqual(numDocs); - } - } - public void testSegmentReplication_Index_Update_Delete() throws Exception { String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; - try (ReplicationGroup shards = createGroup(2, settings, mappings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(2, getIndexSettings(), mappings, new NRTReplicationEngineFactory())) { shards.startAll(); final IndexShard primaryShard = shards.getPrimary(); @@ -414,7 +390,7 @@ public void testSegmentReplication_Index_Update_Delete() throws Exception { public void testIgnoreShardIdle() throws Exception { Settings updatedSettings = Settings.builder() - .put(settings) + .put(getIndexSettings()) .put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), TimeValue.ZERO) .build(); try (ReplicationGroup shards = createGroup(1, updatedSettings, new NRTReplicationEngineFactory())) { @@ -464,7 +440,7 @@ public void testShardIdle_Docrep() throws Exception { public void testShardIdleWithNoReplicas() throws Exception { Settings updatedSettings = Settings.builder() - .put(settings) + .put(getIndexSettings()) .put(IndexSettings.INDEX_SEARCH_IDLE_AFTER.getKey(), TimeValue.ZERO) .build(); try (ReplicationGroup shards = createGroup(0, updatedSettings, new NRTReplicationEngineFactory())) { @@ -544,85 +520,8 @@ public void testRejectCheckpointOnShardRoutingPrimary() throws IOException { closeShards(primaryShard); } - public void testReplicaReceivesGenIncrease() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { - shards.startAll(); - final IndexShard primary = shards.getPrimary(); - final IndexShard replica = shards.getReplicas().get(0); - final int numDocs = randomIntBetween(10, 100); - shards.indexDocs(numDocs); - assertEquals(numDocs, primary.translogStats().estimatedNumberOfOperations()); - assertEquals(numDocs, replica.translogStats().estimatedNumberOfOperations()); - assertEquals(numDocs, primary.translogStats().getUncommittedOperations()); - assertEquals(numDocs, replica.translogStats().getUncommittedOperations()); - flushShard(primary, true); - replicateSegments(primary, shards.getReplicas()); - assertEquals(0, primary.translogStats().estimatedNumberOfOperations()); - assertEquals(0, replica.translogStats().estimatedNumberOfOperations()); - assertEquals(0, primary.translogStats().getUncommittedOperations()); - assertEquals(0, replica.translogStats().getUncommittedOperations()); - - final int additionalDocs = shards.indexDocs(randomIntBetween(numDocs + 1, numDocs + 10)); - - final int totalDocs = numDocs + additionalDocs; - primary.refresh("test"); - replicateSegments(primary, shards.getReplicas()); - assertEquals(additionalDocs, primary.translogStats().estimatedNumberOfOperations()); - assertEquals(additionalDocs, replica.translogStats().estimatedNumberOfOperations()); - assertEquals(additionalDocs, primary.translogStats().getUncommittedOperations()); - assertEquals(additionalDocs, replica.translogStats().getUncommittedOperations()); - flushShard(primary, true); - replicateSegments(primary, shards.getReplicas()); - - assertEqualCommittedSegments(primary, replica); - assertDocCount(primary, totalDocs); - assertDocCount(replica, totalDocs); - assertEquals(0, primary.translogStats().estimatedNumberOfOperations()); - assertEquals(0, replica.translogStats().estimatedNumberOfOperations()); - assertEquals(0, primary.translogStats().getUncommittedOperations()); - assertEquals(0, replica.translogStats().getUncommittedOperations()); - } - } - - public void testPrimaryRelocation() throws Exception { - final IndexShard primarySource = newStartedShard(true, settings); - int totalOps = randomInt(10); - for (int i = 0; i < totalOps; i++) { - indexDoc(primarySource, "_doc", Integer.toString(i)); - } - IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1)); - final IndexShard primaryTarget = newShard( - primarySource.routingEntry().getTargetRelocatingShard(), - settings, - new NRTReplicationEngineFactory() - ); - updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetadata()); - - Function, List> replicatePrimaryFunction = (shardList) -> { - try { - assert shardList.size() >= 2; - final IndexShard primary = shardList.get(0); - return replicateSegments(primary, shardList.subList(1, shardList.size())); - } catch (IOException | InterruptedException e) { - throw new RuntimeException(e); - } - }; - recoverReplica(primaryTarget, primarySource, true, replicatePrimaryFunction); - - // check that local checkpoint of new primary is properly tracked after primary relocation - assertThat(primaryTarget.getLocalCheckpoint(), equalTo(totalOps - 1L)); - assertThat( - primaryTarget.getReplicationTracker() - .getTrackedLocalCheckpointForShard(primaryTarget.routingEntry().allocationId().getId()) - .getLocalCheckpoint(), - equalTo(totalOps - 1L) - ); - assertDocCount(primaryTarget, totalOps); - closeShards(primarySource, primaryTarget); - } - public void testPrimaryRelocationWithSegRepFailure() throws Exception { - final IndexShard primarySource = newStartedShard(true, settings); + final IndexShard primarySource = newStartedShard(true, getIndexSettings()); int totalOps = randomInt(10); for (int i = 0; i < totalOps; i++) { indexDoc(primarySource, "_doc", Integer.toString(i)); @@ -630,7 +529,7 @@ public void testPrimaryRelocationWithSegRepFailure() throws Exception { IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1)); final IndexShard primaryTarget = newShard( primarySource.routingEntry().getTargetRelocatingShard(), - settings, + getIndexSettings(), new NRTReplicationEngineFactory() ); updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetadata()); @@ -669,7 +568,7 @@ public void onFailure(ReplicationState state, ReplicationFailedException e, bool // Todo: Remove this test when there is a better mechanism to test a functionality passing in different replication // strategy. public void testLockingBeforeAndAfterRelocated() throws Exception { - final IndexShard shard = newStartedShard(true, settings); + final IndexShard shard = newStartedShard(true, getIndexSettings()); final ShardRouting routing = ShardRoutingHelper.relocate(shard.routingEntry(), "other_node"); IndexShardTestCase.updateRoutingEntry(shard, routing); CountDownLatch latch = new CountDownLatch(1); @@ -702,7 +601,7 @@ public void testLockingBeforeAndAfterRelocated() throws Exception { // Todo: Remove this test when there is a better mechanism to test a functionality passing in different replication // strategy. public void testDelayedOperationsBeforeAndAfterRelocated() throws Exception { - final IndexShard shard = newStartedShard(true, settings); + final IndexShard shard = newStartedShard(true, getIndexSettings()); final ShardRouting routing = ShardRoutingHelper.relocate(shard.routingEntry(), "other_node"); IndexShardTestCase.updateRoutingEntry(shard, routing); final CountDownLatch startRecovery = new CountDownLatch(1); @@ -776,344 +675,8 @@ public void onFailure(Exception e) { closeShards(shard); } - public void testReplicaReceivesLowerGeneration() throws Exception { - // when a replica gets incoming segments that are lower than what it currently has on disk. - - // start 3 nodes Gens: P [2], R [2], R[2] - // index some docs and flush twice, push to only 1 replica. - // State Gens: P [4], R-1 [3], R-2 [2] - // Promote R-2 as the new primary and demote the old primary. - // State Gens: R[4], R-1 [3], P [4] - *commit on close of NRTEngine, xlog replayed and commit made. - // index docs on new primary and flush - // replicate to all. - // Expected result: State Gens: P[4], R-1 [4], R-2 [4] - try (ReplicationGroup shards = createGroup(2, settings, new NRTReplicationEngineFactory())) { - shards.startAll(); - final IndexShard primary = shards.getPrimary(); - final IndexShard replica_1 = shards.getReplicas().get(0); - final IndexShard replica_2 = shards.getReplicas().get(1); - int numDocs = randomIntBetween(10, 100); - shards.indexDocs(numDocs); - flushShard(primary, false); - replicateSegments(primary, List.of(replica_1)); - numDocs = randomIntBetween(numDocs + 1, numDocs + 10); - shards.indexDocs(numDocs); - flushShard(primary, false); - replicateSegments(primary, List.of(replica_1)); - - assertEqualCommittedSegments(primary, replica_1); - - shards.promoteReplicaToPrimary(replica_2).get(); - primary.close("demoted", false, false); - primary.store().close(); - IndexShard oldPrimary = shards.addReplicaWithExistingPath(primary.shardPath(), primary.routingEntry().currentNodeId()); - shards.recoverReplica(oldPrimary); - - numDocs = randomIntBetween(numDocs + 1, numDocs + 10); - shards.indexDocs(numDocs); - flushShard(replica_2, false); - replicateSegments(replica_2, shards.getReplicas()); - assertEqualCommittedSegments(replica_2, oldPrimary, replica_1); - } - } - - public void testReplicaRestarts() throws Exception { - try (ReplicationGroup shards = createGroup(3, settings, new NRTReplicationEngineFactory())) { - shards.startAll(); - IndexShard primary = shards.getPrimary(); - // 1. Create ops that are in the index and xlog of both shards but not yet part of a commit point. - final int numDocs = shards.indexDocs(randomInt(10)); - - // refresh and copy the segments over. - if (randomBoolean()) { - flushShard(primary); - } - primary.refresh("Test"); - replicateSegments(primary, shards.getReplicas()); - - // at this point both shards should have numDocs persisted and searchable. - assertDocCounts(primary, numDocs, numDocs); - for (IndexShard shard : shards.getReplicas()) { - assertDocCounts(shard, numDocs, numDocs); - } - - final int i1 = randomInt(5); - for (int i = 0; i < i1; i++) { - shards.indexDocs(randomInt(10)); - - // randomly resetart a replica - final IndexShard replicaToRestart = getRandomReplica(shards); - replicaToRestart.close("restart", false, false); - replicaToRestart.store().close(); - shards.removeReplica(replicaToRestart); - final IndexShard newReplica = shards.addReplicaWithExistingPath( - replicaToRestart.shardPath(), - replicaToRestart.routingEntry().currentNodeId() - ); - shards.recoverReplica(newReplica); - - // refresh and push segments to our other replicas. - if (randomBoolean()) { - failAndPromoteRandomReplica(shards); - } - flushShard(shards.getPrimary()); - replicateSegments(shards.getPrimary(), shards.getReplicas()); - } - primary = shards.getPrimary(); - - // refresh and push segments to our other replica. - flushShard(primary); - replicateSegments(primary, shards.getReplicas()); - - for (IndexShard shard : shards) { - assertConsistentHistoryBetweenTranslogAndLucene(shard); - } - final List docsAfterReplication = getDocIdAndSeqNos(shards.getPrimary()); - for (IndexShard shard : shards.getReplicas()) { - assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterReplication)); - } - } - } - - public void testNRTReplicaWithRemoteStorePromotedAsPrimaryRefreshRefresh() throws Exception { - testNRTReplicaWithRemoteStorePromotedAsPrimary(false, false); - } - - public void testNRTReplicaWithRemoteStorePromotedAsPrimaryRefreshCommit() throws Exception { - testNRTReplicaWithRemoteStorePromotedAsPrimary(false, true); - } - - public void testNRTReplicaWithRemoteStorePromotedAsPrimaryCommitRefresh() throws Exception { - testNRTReplicaWithRemoteStorePromotedAsPrimary(true, false); - } - - public void testNRTReplicaWithRemoteStorePromotedAsPrimaryCommitCommit() throws Exception { - testNRTReplicaWithRemoteStorePromotedAsPrimary(true, true); - } - - private void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlushFirst, boolean performFlushSecond) throws Exception { - Settings settings = Settings.builder() - .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) - .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) - .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, "temp-fs") - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "temp-fs") - .build(); - - try (ReplicationGroup shards = createGroup(1, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir())) { - shards.startAll(); - IndexShard oldPrimary = shards.getPrimary(); - final IndexShard nextPrimary = shards.getReplicas().get(0); - - // 1. Create ops that are in the index and xlog of both shards but not yet part of a commit point. - final int numDocs = shards.indexDocs(randomInt(10)); - - // refresh but do not copy the segments over. - if (performFlushFirst) { - flushShard(oldPrimary, true); - } else { - oldPrimary.refresh("Test"); - } - // replicateSegments(primary, shards.getReplicas()); - - // at this point both shards should have numDocs persisted and searchable. - assertDocCounts(oldPrimary, numDocs, numDocs); - for (IndexShard shard : shards.getReplicas()) { - assertDocCounts(shard, numDocs, 0); - } - - // 2. Create ops that are in the replica's xlog, not in the index. - // index some more into both but don't replicate. replica will have only numDocs searchable, but should have totalDocs - // persisted. - final int additonalDocs = shards.indexDocs(randomInt(10)); - final int totalDocs = numDocs + additonalDocs; - - if (performFlushSecond) { - flushShard(oldPrimary, true); - } else { - oldPrimary.refresh("Test"); - } - assertDocCounts(oldPrimary, totalDocs, totalDocs); - for (IndexShard shard : shards.getReplicas()) { - assertDocCounts(shard, totalDocs, 0); - } - assertTrue(nextPrimary.translogStats().estimatedNumberOfOperations() >= additonalDocs); - assertTrue(nextPrimary.translogStats().getUncommittedOperations() >= additonalDocs); - - int prevOperationCount = nextPrimary.translogStats().estimatedNumberOfOperations(); - - // promote the replica - shards.promoteReplicaToPrimary(nextPrimary).get(); - - // close oldPrimary. - oldPrimary.close("demoted", false, false); - oldPrimary.store().close(); - - assertEquals(InternalEngine.class, nextPrimary.getEngine().getClass()); - assertDocCounts(nextPrimary, totalDocs, totalDocs); - - // As we are downloading segments from remote segment store on failover, there should not be - // any operations replayed from translog - assertEquals(prevOperationCount, nextPrimary.translogStats().estimatedNumberOfOperations()); - - // refresh and push segments to our other replica. - nextPrimary.refresh("test"); - - for (IndexShard shard : shards) { - assertConsistentHistoryBetweenTranslogAndLucene(shard); - } - final List docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary()); - for (IndexShard shard : shards.getReplicas()) { - assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery)); - } - } - } - - public void testNRTReplicaPromotedAsPrimary() throws Exception { - try (ReplicationGroup shards = createGroup(2, settings, new NRTReplicationEngineFactory())) { - shards.startAll(); - IndexShard oldPrimary = shards.getPrimary(); - final IndexShard nextPrimary = shards.getReplicas().get(0); - final IndexShard replica = shards.getReplicas().get(1); - - // 1. Create ops that are in the index and xlog of both shards but not yet part of a commit point. - final int numDocs = shards.indexDocs(randomInt(10)); - - // refresh and copy the segments over. - oldPrimary.refresh("Test"); - replicateSegments(oldPrimary, shards.getReplicas()); - - // at this point both shards should have numDocs persisted and searchable. - assertDocCounts(oldPrimary, numDocs, numDocs); - for (IndexShard shard : shards.getReplicas()) { - assertDocCounts(shard, numDocs, numDocs); - } - assertEqualTranslogOperations(shards, oldPrimary); - - // 2. Create ops that are in the replica's xlog, not in the index. - // index some more into both but don't replicate. replica will have only numDocs searchable, but should have totalDocs - // persisted. - final int additonalDocs = shards.indexDocs(randomInt(10)); - final int totalDocs = numDocs + additonalDocs; - - assertDocCounts(oldPrimary, totalDocs, totalDocs); - assertEqualTranslogOperations(shards, oldPrimary); - for (IndexShard shard : shards.getReplicas()) { - assertDocCounts(shard, totalDocs, numDocs); - } - assertEquals(totalDocs, oldPrimary.translogStats().estimatedNumberOfOperations()); - assertEquals(totalDocs, oldPrimary.translogStats().estimatedNumberOfOperations()); - assertEquals(totalDocs, nextPrimary.translogStats().estimatedNumberOfOperations()); - assertEquals(totalDocs, replica.translogStats().estimatedNumberOfOperations()); - assertEquals(totalDocs, nextPrimary.translogStats().getUncommittedOperations()); - assertEquals(totalDocs, replica.translogStats().getUncommittedOperations()); - - // promote the replica - shards.syncGlobalCheckpoint(); - shards.promoteReplicaToPrimary(nextPrimary); - - // close and start the oldPrimary as a replica. - oldPrimary.close("demoted", false, false); - oldPrimary.store().close(); - oldPrimary = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); - shards.recoverReplica(oldPrimary); - - assertEquals(NRTReplicationEngine.class, oldPrimary.getEngine().getClass()); - assertEquals(InternalEngine.class, nextPrimary.getEngine().getClass()); - assertDocCounts(nextPrimary, totalDocs, totalDocs); - assertEquals(0, nextPrimary.translogStats().estimatedNumberOfOperations()); - - // refresh and push segments to our other replica. - nextPrimary.refresh("test"); - replicateSegments(nextPrimary, asList(replica)); - - for (IndexShard shard : shards) { - assertConsistentHistoryBetweenTranslogAndLucene(shard); - } - final List docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary()); - for (IndexShard shard : shards.getReplicas()) { - assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery)); - } - } - } - - public void testReplicaPromotedWhileReplicating() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { - shards.startAll(); - final IndexShard oldPrimary = shards.getPrimary(); - final IndexShard nextPrimary = shards.getReplicas().get(0); - - final int numDocs = shards.indexDocs(randomInt(10)); - oldPrimary.refresh("Test"); - shards.syncGlobalCheckpoint(); - - final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); - final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); - SegmentReplicationSource source = new TestReplicationSource() { - @Override - public void getCheckpointMetadata( - long replicationId, - ReplicationCheckpoint checkpoint, - ActionListener listener - ) { - resolveCheckpointInfoResponseListener(listener, oldPrimary); - ShardRouting oldRouting = nextPrimary.shardRouting; - try { - shards.promoteReplicaToPrimary(nextPrimary); - } catch (IOException e) { - Assert.fail("Promotion should not fail"); - } - targetService.shardRoutingChanged(nextPrimary, oldRouting, nextPrimary.shardRouting); - } - - @Override - public void getSegmentFiles( - long replicationId, - ReplicationCheckpoint checkpoint, - List filesToFetch, - IndexShard indexShard, - ActionListener listener - ) { - listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); - } - }; - when(sourceFactory.get(any())).thenReturn(source); - startReplicationAndAssertCancellation(nextPrimary, targetService); - // wait for replica to finish being promoted, and assert doc counts. - final CountDownLatch latch = new CountDownLatch(1); - nextPrimary.acquirePrimaryOperationPermit(new ActionListener<>() { - @Override - public void onResponse(Releasable releasable) { - latch.countDown(); - } - - @Override - public void onFailure(Exception e) { - throw new AssertionError(e); - } - }, ThreadPool.Names.GENERIC, ""); - latch.await(); - assertEquals(nextPrimary.getEngine().getClass(), InternalEngine.class); - nextPrimary.refresh("test"); - - oldPrimary.close("demoted", false, false); - oldPrimary.store().close(); - IndexShard newReplica = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); - shards.recoverReplica(newReplica); - - assertDocCount(nextPrimary, numDocs); - assertDocCount(newReplica, numDocs); - - nextPrimary.refresh("test"); - replicateSegments(nextPrimary, shards.getReplicas()); - final List docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary()); - for (IndexShard shard : shards.getReplicas()) { - assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery)); - } - } - } - public void testReplicaClosesWhileReplicating_AfterGetCheckpoint() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); @@ -1155,7 +718,7 @@ public void getSegmentFiles( } public void testReplicaClosesWhileReplicating_AfterGetSegmentFiles() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); @@ -1197,7 +760,7 @@ public void getSegmentFiles( } public void testCloseShardDuringFinalize() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); @@ -1212,7 +775,7 @@ public void testCloseShardDuringFinalize() throws Exception { } public void testCloseShardWhileGettingCheckpoint() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); @@ -1264,7 +827,7 @@ public void cancel() { } public void testBeforeIndexShardClosedWhileCopyingFiles() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); @@ -1316,7 +879,7 @@ public void cancel() { } public void testPrimaryCancelsExecution() throws Exception { - try (ReplicationGroup shards = createGroup(1, settings, new NRTReplicationEngineFactory())) { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); IndexShard primary = shards.getPrimary(); final IndexShard replica = shards.getReplicas().get(0); @@ -1353,7 +916,7 @@ public void getSegmentFiles( } } - private SegmentReplicationTargetService newTargetService(SegmentReplicationSourceFactory sourceFactory) { + protected SegmentReplicationTargetService newTargetService(SegmentReplicationSourceFactory sourceFactory) { return new SegmentReplicationTargetService( threadPool, new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), @@ -1368,14 +931,15 @@ private SegmentReplicationTargetService newTargetService(SegmentReplicationSourc * Assert persisted and searchable doc counts. This method should not be used while docs are concurrently indexed because * it asserts point in time seqNos are relative to the doc counts. */ - private void assertDocCounts(IndexShard indexShard, int expectedPersistedDocCount, int expectedSearchableDocCount) throws IOException { + protected void assertDocCounts(IndexShard indexShard, int expectedPersistedDocCount, int expectedSearchableDocCount) + throws IOException { assertDocCount(indexShard, expectedSearchableDocCount); // assigned seqNos start at 0, so assert max & local seqNos are 1 less than our persisted doc count. assertEquals(expectedPersistedDocCount - 1, indexShard.seqNoStats().getMaxSeqNo()); assertEquals(expectedPersistedDocCount - 1, indexShard.seqNoStats().getLocalCheckpoint()); } - private void resolveCheckpointInfoResponseListener(ActionListener listener, IndexShard primary) { + protected void resolveCheckpointInfoResponseListener(ActionListener listener, IndexShard primary) { try { final CopyState copyState = new CopyState( ReplicationCheckpoint.empty(primary.shardId, primary.getLatestReplicationCheckpoint().getCodec()), @@ -1390,7 +954,7 @@ private void resolveCheckpointInfoResponseListener(ActionListener operations = new ArrayList<>(); Translog.Operation op; diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java new file mode 100644 index 0000000000000..dc39b0c27ae1e --- /dev/null +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java @@ -0,0 +1,469 @@ +/* + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.index.shard; + +import org.apache.lucene.index.SegmentInfos; +import org.junit.Assert; +import org.opensearch.action.ActionListener; +import org.opensearch.action.admin.indices.flush.FlushRequest; +import org.opensearch.action.index.IndexRequest; +import org.opensearch.cluster.routing.ShardRouting; +import org.opensearch.common.concurrent.GatedCloseable; +import org.opensearch.common.lease.Releasable; +import org.opensearch.common.settings.ClusterSettings; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.xcontent.XContentType; +import org.opensearch.index.engine.DocIdSeqNoAndSource; +import org.opensearch.index.engine.InternalEngine; +import org.opensearch.index.engine.NRTReplicationEngine; +import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.mapper.MapperService; +import org.opensearch.index.replication.TestReplicationSource; +import org.opensearch.index.store.StoreFileMetadata; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.CheckpointInfoResponse; +import org.opensearch.indices.replication.GetSegmentFilesResponse; +import org.opensearch.indices.replication.SegmentReplicationSource; +import org.opensearch.indices.replication.SegmentReplicationSourceFactory; +import org.opensearch.indices.replication.SegmentReplicationTarget; +import org.opensearch.indices.replication.SegmentReplicationTargetService; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.threadpool.ThreadPool; +import org.opensearch.transport.TransportService; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static java.util.Arrays.asList; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class SegmentReplicationWithNodeToNodeIndexShardTests extends SegmentReplicationIndexShardTests { + + public void testReplicaPromotedWhileReplicating() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + final IndexShard oldPrimary = shards.getPrimary(); + final IndexShard nextPrimary = shards.getReplicas().get(0); + + final int numDocs = shards.indexDocs(randomInt(10)); + oldPrimary.refresh("Test"); + shards.syncGlobalCheckpoint(); + + final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); + final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); + SegmentReplicationSource source = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) { + resolveCheckpointInfoResponseListener(listener, oldPrimary); + ShardRouting oldRouting = nextPrimary.shardRouting; + try { + shards.promoteReplicaToPrimary(nextPrimary); + } catch (IOException e) { + Assert.fail("Promotion should not fail"); + } + targetService.shardRoutingChanged(nextPrimary, oldRouting, nextPrimary.shardRouting); + } + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + ActionListener listener + ) { + listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); + } + }; + when(sourceFactory.get(any())).thenReturn(source); + startReplicationAndAssertCancellation(nextPrimary, targetService); + // wait for replica to finish being promoted, and assert doc counts. + final CountDownLatch latch = new CountDownLatch(1); + nextPrimary.acquirePrimaryOperationPermit(new ActionListener<>() { + @Override + public void onResponse(Releasable releasable) { + latch.countDown(); + } + + @Override + public void onFailure(Exception e) { + throw new AssertionError(e); + } + }, ThreadPool.Names.GENERIC, ""); + latch.await(); + assertEquals(nextPrimary.getEngine().getClass(), InternalEngine.class); + nextPrimary.refresh("test"); + + oldPrimary.close("demoted", false, false); + oldPrimary.store().close(); + IndexShard newReplica = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); + shards.recoverReplica(newReplica); + + assertDocCount(nextPrimary, numDocs); + assertDocCount(newReplica, numDocs); + + nextPrimary.refresh("test"); + replicateSegments(nextPrimary, shards.getReplicas()); + final List docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary()); + for (IndexShard shard : shards.getReplicas()) { + assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery)); + } + } + } + + public void testReplicaReceivesGenIncrease() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + final IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + final int numDocs = randomIntBetween(10, 100); + shards.indexDocs(numDocs); + assertEquals(numDocs, primary.translogStats().estimatedNumberOfOperations()); + assertEquals(numDocs, replica.translogStats().estimatedNumberOfOperations()); + assertEquals(numDocs, primary.translogStats().getUncommittedOperations()); + assertEquals(numDocs, replica.translogStats().getUncommittedOperations()); + flushShard(primary, true); + replicateSegments(primary, shards.getReplicas()); + assertEquals(0, primary.translogStats().estimatedNumberOfOperations()); + assertEquals(0, replica.translogStats().estimatedNumberOfOperations()); + assertEquals(0, primary.translogStats().getUncommittedOperations()); + assertEquals(0, replica.translogStats().getUncommittedOperations()); + + final int additionalDocs = shards.indexDocs(randomIntBetween(numDocs + 1, numDocs + 10)); + + final int totalDocs = numDocs + additionalDocs; + primary.refresh("test"); + replicateSegments(primary, shards.getReplicas()); + assertEquals(additionalDocs, primary.translogStats().estimatedNumberOfOperations()); + assertEquals(additionalDocs, replica.translogStats().estimatedNumberOfOperations()); + assertEquals(additionalDocs, primary.translogStats().getUncommittedOperations()); + assertEquals(additionalDocs, replica.translogStats().getUncommittedOperations()); + flushShard(primary, true); + replicateSegments(primary, shards.getReplicas()); + + assertEqualCommittedSegments(primary, replica); + assertDocCount(primary, totalDocs); + assertDocCount(replica, totalDocs); + assertEquals(0, primary.translogStats().estimatedNumberOfOperations()); + assertEquals(0, replica.translogStats().estimatedNumberOfOperations()); + assertEquals(0, primary.translogStats().getUncommittedOperations()); + assertEquals(0, replica.translogStats().getUncommittedOperations()); + } + } + + /** + * Verifies that commits on replica engine resulting from engine or reader close does not cleanup the temporary + * replication files from ongoing round of segment replication + * @throws Exception + */ + public void testTemporaryFilesNotCleanup() throws Exception { + String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), mappings, new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard primaryShard = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + + // Step 1. Ingest numDocs documents, commit to create commit point on primary & replicate + final int numDocs = randomIntBetween(100, 200); + logger.info("--> Inserting documents {}", numDocs); + for (int i = 0; i < numDocs; i++) { + shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); + } + assertEqualTranslogOperations(shards, primaryShard); + primaryShard.flush(new FlushRequest().waitIfOngoing(true).force(true)); + replicateSegments(primaryShard, shards.getReplicas()); + shards.assertAllEqual(numDocs); + + // Step 2. Ingest numDocs documents again to create a new commit on primary + logger.info("--> Ingest {} docs again", numDocs); + for (int i = 0; i < numDocs; i++) { + shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); + } + assertEqualTranslogOperations(shards, primaryShard); + primaryShard.flush(new FlushRequest().waitIfOngoing(true).force(true)); + + // Step 3. Copy segment files to replica shard but prevent commit + final CountDownLatch countDownLatch = new CountDownLatch(1); + Map primaryMetadata; + try (final GatedCloseable segmentInfosSnapshot = primaryShard.getSegmentInfosSnapshot()) { + final SegmentInfos primarySegmentInfos = segmentInfosSnapshot.get(); + primaryMetadata = primaryShard.store().getSegmentMetadataMap(primarySegmentInfos); + } + final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); + final IndicesService indicesService = mock(IndicesService.class); + when(indicesService.getShardOrNull(replica.shardId)).thenReturn(replica); + final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( + threadPool, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + mock(TransportService.class), + sourceFactory, + indicesService, + clusterService + ); + final Consumer runnablePostGetFiles = (indexShard) -> { + try { + Collection temporaryFiles = Stream.of(indexShard.store().directory().listAll()) + .filter(name -> name.startsWith(SegmentReplicationTarget.REPLICATION_PREFIX)) + .collect(Collectors.toList()); + + // Step 4. Perform a commit on replica shard. + NRTReplicationEngine engine = (NRTReplicationEngine) indexShard.getEngine(); + engine.updateSegments(engine.getSegmentInfosSnapshot().get()); + + // Step 5. Validate temporary files are not deleted from store. + Collection replicaStoreFiles = List.of(indexShard.store().directory().listAll()); + assertTrue(replicaStoreFiles.containsAll(temporaryFiles)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }; + SegmentReplicationSource segmentReplicationSource = getSegmentReplicationSource( + primaryShard, + (repId) -> targetService.get(repId), + runnablePostGetFiles + ); + when(sourceFactory.get(any())).thenReturn(segmentReplicationSource); + targetService.startReplication(replica, getTargetListener(primaryShard, replica, primaryMetadata, countDownLatch)); + countDownLatch.await(30, TimeUnit.SECONDS); + assertEquals("Replication failed", 0, countDownLatch.getCount()); + shards.assertAllEqual(numDocs); + } + } + + public void testReplicaReceivesLowerGeneration() throws Exception { + // when a replica gets incoming segments that are lower than what it currently has on disk. + + // start 3 nodes Gens: P [2], R [2], R[2] + // index some docs and flush twice, push to only 1 replica. + // State Gens: P [4], R-1 [3], R-2 [2] + // Promote R-2 as the new primary and demote the old primary. + // State Gens: R[4], R-1 [3], P [4] - *commit on close of NRTEngine, xlog replayed and commit made. + // index docs on new primary and flush + // replicate to all. + // Expected result: State Gens: P[4], R-1 [4], R-2 [4] + try (ReplicationGroup shards = createGroup(2, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + final IndexShard primary = shards.getPrimary(); + final IndexShard replica_1 = shards.getReplicas().get(0); + final IndexShard replica_2 = shards.getReplicas().get(1); + int numDocs = randomIntBetween(10, 100); + shards.indexDocs(numDocs); + flushShard(primary, false); + replicateSegments(primary, List.of(replica_1)); + numDocs = randomIntBetween(numDocs + 1, numDocs + 10); + shards.indexDocs(numDocs); + flushShard(primary, false); + replicateSegments(primary, List.of(replica_1)); + + assertEqualCommittedSegments(primary, replica_1); + + shards.promoteReplicaToPrimary(replica_2).get(); + primary.close("demoted", false, false); + primary.store().close(); + IndexShard oldPrimary = shards.addReplicaWithExistingPath(primary.shardPath(), primary.routingEntry().currentNodeId()); + shards.recoverReplica(oldPrimary); + + numDocs = randomIntBetween(numDocs + 1, numDocs + 10); + shards.indexDocs(numDocs); + flushShard(replica_2, false); + replicateSegments(replica_2, shards.getReplicas()); + assertEqualCommittedSegments(replica_2, oldPrimary, replica_1); + } + } + + // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store + public void testPrimaryRelocation() throws Exception { + final IndexShard primarySource = newStartedShard(true, getIndexSettings()); + int totalOps = randomInt(10); + for (int i = 0; i < totalOps; i++) { + indexDoc(primarySource, "_doc", Integer.toString(i)); + } + IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1)); + final IndexShard primaryTarget = newShard( + primarySource.routingEntry().getTargetRelocatingShard(), + getIndexSettings(), + new NRTReplicationEngineFactory() + ); + updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetadata()); + + Function, List> replicatePrimaryFunction = (shardList) -> { + try { + assert shardList.size() >= 2; + final IndexShard primary = shardList.get(0); + return replicateSegments(primary, shardList.subList(1, shardList.size())); + } catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + }; + recoverReplica(primaryTarget, primarySource, true, replicatePrimaryFunction); + + // check that local checkpoint of new primary is properly tracked after primary relocation + assertThat(primaryTarget.getLocalCheckpoint(), equalTo(totalOps - 1L)); + assertThat( + primaryTarget.getReplicationTracker() + .getTrackedLocalCheckpointForShard(primaryTarget.routingEntry().allocationId().getId()) + .getLocalCheckpoint(), + equalTo(totalOps - 1L) + ); + assertDocCount(primaryTarget, totalOps); + closeShards(primarySource, primaryTarget); + } + + // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store + public void testNRTReplicaPromotedAsPrimary() throws Exception { + try (ReplicationGroup shards = createGroup(2, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard oldPrimary = shards.getPrimary(); + final IndexShard nextPrimary = shards.getReplicas().get(0); + final IndexShard replica = shards.getReplicas().get(1); + + // 1. Create ops that are in the index and xlog of both shards but not yet part of a commit point. + final int numDocs = shards.indexDocs(randomInt(10)); + + // refresh and copy the segments over. + oldPrimary.refresh("Test"); + replicateSegments(oldPrimary, shards.getReplicas()); + + // at this point both shards should have numDocs persisted and searchable. + assertDocCounts(oldPrimary, numDocs, numDocs); + for (IndexShard shard : shards.getReplicas()) { + assertDocCounts(shard, numDocs, numDocs); + } + assertEqualTranslogOperations(shards, oldPrimary); + + // 2. Create ops that are in the replica's xlog, not in the index. + // index some more into both but don't replicate. replica will have only numDocs searchable, but should have totalDocs + // persisted. + final int additonalDocs = shards.indexDocs(randomInt(10)); + final int totalDocs = numDocs + additonalDocs; + + assertDocCounts(oldPrimary, totalDocs, totalDocs); + assertEqualTranslogOperations(shards, oldPrimary); + for (IndexShard shard : shards.getReplicas()) { + assertDocCounts(shard, totalDocs, numDocs); + } + assertEquals(totalDocs, oldPrimary.translogStats().estimatedNumberOfOperations()); + assertEquals(totalDocs, oldPrimary.translogStats().estimatedNumberOfOperations()); + assertEquals(totalDocs, nextPrimary.translogStats().estimatedNumberOfOperations()); + assertEquals(totalDocs, replica.translogStats().estimatedNumberOfOperations()); + assertEquals(totalDocs, nextPrimary.translogStats().getUncommittedOperations()); + assertEquals(totalDocs, replica.translogStats().getUncommittedOperations()); + + // promote the replica + shards.syncGlobalCheckpoint(); + shards.promoteReplicaToPrimary(nextPrimary); + + // close and start the oldPrimary as a replica. + oldPrimary.close("demoted", false, false); + oldPrimary.store().close(); + oldPrimary = shards.addReplicaWithExistingPath(oldPrimary.shardPath(), oldPrimary.routingEntry().currentNodeId()); + shards.recoverReplica(oldPrimary); + + assertEquals(NRTReplicationEngine.class, oldPrimary.getEngine().getClass()); + assertEquals(InternalEngine.class, nextPrimary.getEngine().getClass()); + assertDocCounts(nextPrimary, totalDocs, totalDocs); + assertEquals(0, nextPrimary.translogStats().estimatedNumberOfOperations()); + + // refresh and push segments to our other replica. + nextPrimary.refresh("test"); + replicateSegments(nextPrimary, asList(replica)); + + for (IndexShard shard : shards) { + assertConsistentHistoryBetweenTranslogAndLucene(shard); + } + final List docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary()); + for (IndexShard shard : shards.getReplicas()) { + assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery)); + } + } + } + + // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store + public void testReplicaRestarts() throws Exception { + try (ReplicationGroup shards = createGroup(3, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard primary = shards.getPrimary(); + // 1. Create ops that are in the index and xlog of both shards but not yet part of a commit point. + final int numDocs = shards.indexDocs(randomInt(10)); + logger.info("--> Index {} documents on primary", numDocs); + + // refresh and copy the segments over. + if (randomBoolean()) { + flushShard(primary); + } + primary.refresh("Test"); + logger.info("--> Replicate segments"); + replicateSegments(primary, shards.getReplicas()); + + // at this point both shards should have numDocs persisted and searchable. + logger.info("--> Verify doc count"); + assertDocCounts(primary, numDocs, numDocs); + for (IndexShard shard : shards.getReplicas()) { + assertDocCounts(shard, numDocs, numDocs); + } + + final int i1 = randomInt(5); + logger.info("--> Index {} more docs", i1); + for (int i = 0; i < i1; i++) { + shards.indexDocs(randomInt(10)); + + // randomly restart a replica + final IndexShard replicaToRestart = getRandomReplica(shards); + logger.info("--> Restarting replica {}", replicaToRestart.shardId); + replicaToRestart.close("restart", false, false); + replicaToRestart.store().close(); + shards.removeReplica(replicaToRestart); + final IndexShard newReplica = shards.addReplicaWithExistingPath( + replicaToRestart.shardPath(), + replicaToRestart.routingEntry().currentNodeId() + ); + logger.info("--> Recover newReplica {}", newReplica.shardId); + shards.recoverReplica(newReplica); + + // refresh and push segments to our other replicas. + if (randomBoolean()) { + failAndPromoteRandomReplica(shards); + } + flushShard(shards.getPrimary()); + replicateSegments(shards.getPrimary(), shards.getReplicas()); + } + primary = shards.getPrimary(); + + // refresh and push segments to our other replica. + flushShard(primary); + replicateSegments(primary, shards.getReplicas()); + + for (IndexShard shard : shards) { + assertConsistentHistoryBetweenTranslogAndLucene(shard); + } + final List docsAfterReplication = getDocIdAndSeqNos(shards.getPrimary()); + for (IndexShard shard : shards.getReplicas()) { + assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterReplication)); + } + } + } + +} diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java index d522e21261b37..b67e7b34b02a5 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java @@ -8,36 +8,168 @@ package org.opensearch.index.shard; +import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.engine.DocIdSeqNoAndSource; +import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; -import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.replication.SegmentReplicationSourceFactory; +import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.transport.TransportService; import java.io.IOException; +import java.util.List; +import java.util.function.Consumer; -public class SegmentReplicationWithRemoteIndexShardTests extends OpenSearchIndexLevelReplicationTestCase { +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; + +public class SegmentReplicationWithRemoteIndexShardTests extends SegmentReplicationIndexShardTests { + + private static final String REPOSITORY_NAME = "temp-fs"; private static final Settings settings = Settings.builder() .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) - .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, "temp-fs") - .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, "temp-fs") + .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, REPOSITORY_NAME) + .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, REPOSITORY_NAME) .build(); + TransportService transportService; + IndicesService indicesService; + RecoverySettings recoverySettings; + SegmentReplicationSourceFactory sourceFactory; + + @Before + public void setup() { + recoverySettings = new RecoverySettings( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + transportService = mock(TransportService.class); + indicesService = mock(IndicesService.class); + sourceFactory = new SegmentReplicationSourceFactory(transportService, recoverySettings, clusterService); + // Todo: Remove feature flag once remote store integration with segrep goes GA + FeatureFlags.initializeFeatureFlags( + Settings.builder().put(FeatureFlags.SEGMENT_REPLICATION_EXPERIMENTAL_SETTING.getKey(), "true").build() + ); + } + + protected Settings getIndexSettings() { + return settings; + } + + protected ReplicationGroup getReplicationGroup(int numberOfReplicas) throws IOException { + return createGroup(numberOfReplicas, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir()); + } + + protected SegmentReplicationTargetService prepareForReplication( + IndexShard primaryShard, + IndexShard target, + TransportService transportService, + IndicesService indicesService, + ClusterService clusterService, + Consumer postGetFilesRunnable + ) { + final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( + threadPool, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + transportService, + sourceFactory, + indicesService, + clusterService + ); + return targetService; + } + + public void testNRTReplicaWithRemoteStorePromotedAsPrimaryRefreshRefresh() throws Exception { + testNRTReplicaWithRemoteStorePromotedAsPrimary(false, false); + } + + public void testNRTReplicaWithRemoteStorePromotedAsPrimaryRefreshCommit() throws Exception { + testNRTReplicaWithRemoteStorePromotedAsPrimary(false, true); + } + + public void testNRTReplicaWithRemoteStorePromotedAsPrimaryCommitRefresh() throws Exception { + testNRTReplicaWithRemoteStorePromotedAsPrimary(true, false); + } + + public void testNRTReplicaWithRemoteStorePromotedAsPrimaryCommitCommit() throws Exception { + testNRTReplicaWithRemoteStorePromotedAsPrimary(true, true); + } + + public void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlushFirst, boolean performFlushSecond) throws Exception { + try ( + ReplicationGroup shards = createGroup(1, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory(), createTempDir()) + ) { + shards.startAll(); + IndexShard oldPrimary = shards.getPrimary(); + final IndexShard nextPrimary = shards.getReplicas().get(0); + + // 1. Create ops that are in the index and xlog of both shards but not yet part of a commit point. + final int numDocs = shards.indexDocs(randomInt(10)); + + // refresh but do not copy the segments over. + if (performFlushFirst) { + flushShard(oldPrimary, true); + } else { + oldPrimary.refresh("Test"); + } + // replicateSegments(primary, shards.getReplicas()); + + // at this point both shards should have numDocs persisted and searchable. + assertDocCounts(oldPrimary, numDocs, numDocs); + for (IndexShard shard : shards.getReplicas()) { + assertDocCounts(shard, numDocs, 0); + } + + // 2. Create ops that are in the replica's xlog, not in the index. + // index some more into both but don't replicate. replica will have only numDocs searchable, but should have totalDocs + // persisted. + final int additonalDocs = shards.indexDocs(randomInt(10)); + final int totalDocs = numDocs + additonalDocs; + + if (performFlushSecond) { + flushShard(oldPrimary, true); + } else { + oldPrimary.refresh("Test"); + } + assertDocCounts(oldPrimary, totalDocs, totalDocs); + for (IndexShard shard : shards.getReplicas()) { + assertDocCounts(shard, totalDocs, 0); + } + assertTrue(nextPrimary.translogStats().estimatedNumberOfOperations() >= additonalDocs); + assertTrue(nextPrimary.translogStats().getUncommittedOperations() >= additonalDocs); + + // promote the replica + shards.promoteReplicaToPrimary(nextPrimary).get(); + + // close oldPrimary. + oldPrimary.close("demoted", false, false); + oldPrimary.store().close(); + + assertEquals(InternalEngine.class, nextPrimary.getEngine().getClass()); + assertDocCounts(nextPrimary, totalDocs, totalDocs); + + // As we are downloading segments from remote segment store on failover, there should not be + // any operations replayed from translog + assertEquals(0, nextPrimary.translogStats().estimatedNumberOfOperations()); + + // refresh and push segments to our other replica. + nextPrimary.refresh("test"); - public void testReplicaSyncingFromRemoteStore() throws IOException { - ReplicationGroup shards = createGroup(1, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir()); - final IndexShard primaryShard = shards.getPrimary(); - final IndexShard replicaShard = shards.getReplicas().get(0); - shards.startPrimary(); - shards.startAll(); - indexDoc(primaryShard, "_doc", "1"); - indexDoc(primaryShard, "_doc", "2"); - primaryShard.refresh("test"); - assertDocs(primaryShard, "1", "2"); - flushShard(primaryShard); - - replicaShard.syncSegmentsFromRemoteSegmentStore(true, true); - assertDocs(replicaShard, "1", "2"); - closeShards(primaryShard, replicaShard); + for (IndexShard shard : shards) { + assertConsistentHistoryBetweenTranslogAndLucene(shard); + } + final List docsAfterRecovery = getDocIdAndSeqNos(shards.getPrimary()); + for (IndexShard shard : shards.getReplicas()) { + assertThat(shard.routingEntry().toString(), getDocIdAndSeqNos(shard), equalTo(docsAfterRecovery)); + } + } } } diff --git a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java index f3c98ce4f9f03..b7e1460e22443 100644 --- a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java @@ -149,7 +149,12 @@ protected ReplicationGroup createGroup(int replicas, Settings settings, EngineFa protected ReplicationGroup createGroup(int replicas, Settings settings, String mappings, EngineFactory engineFactory) throws IOException { - return createGroup(replicas, settings, mappings, engineFactory, null); + Path remotePath = null; + if (settings.get(IndexMetadata.SETTING_REMOTE_STORE_ENABLED) != null + && settings.get(IndexMetadata.SETTING_REMOTE_STORE_ENABLED).equals("true")) { + remotePath = createTempDir(); + } + return createGroup(replicas, settings, mappings, engineFactory, remotePath); } protected ReplicationGroup createGroup(int replicas, Settings settings, String mappings, EngineFactory engineFactory, Path remotePath) diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 7a492dbebd836..7a469819116d6 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -1328,7 +1328,9 @@ public static Engine.Warmer createTestWarmer(IndexSettings indexSettings) { /** * Segment Replication specific test method - Creates a {@link SegmentReplicationTargetService} to perform replications that has - * been configured to return the given primaryShard's current segments. + * been configured to return the given primaryShard's current segments. In order to do so, it mimics the replication + * source (to avoid transport calls) and simply copies over the segment files from primary store to replica's as part of + * get_files calls. * * @param primaryShard {@link IndexShard} - The target replica shard in segment replication. * @param target {@link IndexShard} - The source primary shard in segment replication. @@ -1339,7 +1341,7 @@ public static Engine.Warmer createTestWarmer(IndexSettings indexSettings) { * which are desired right after files are copied. e.g. To work with temp files * @return Returns SegmentReplicationTargetService */ - public final SegmentReplicationTargetService prepareForReplication( + protected SegmentReplicationTargetService prepareForReplication( IndexShard primaryShard, IndexShard target, TransportService transportService, @@ -1362,6 +1364,7 @@ public final SegmentReplicationTargetService prepareForReplication( postGetFilesRunnable ); when(sourceFactory.get(any())).thenReturn(replicationSource); + // This is needed for force segment sync call. Remote store uses a different recovery mechanism when(indicesService.getShardOrNull(any())).thenReturn(target); return targetService; } @@ -1502,9 +1505,11 @@ public void getSegmentFiles( * @param replicaShards - Replicas that will be updated. * @return {@link List} List of target components orchestrating replication. */ - public final List replicateSegments(IndexShard primaryShard, List replicaShards) + protected final List replicateSegments(IndexShard primaryShard, List replicaShards) throws IOException, InterruptedException { + // Latch to block test execution until replica catches up final CountDownLatch countDownLatch = new CountDownLatch(replicaShards.size()); + // Get primary metadata to verify with replica's, used to ensure replica catches up Map primaryMetadata; try (final GatedCloseable segmentInfosSnapshot = primaryShard.getSegmentInfosSnapshot()) { final SegmentInfos primarySegmentInfos = segmentInfosSnapshot.get(); From 5d9063a81cb831edf117d98885185326c4010c1e Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Sun, 23 Jul 2023 17:39:41 -0700 Subject: [PATCH 06/26] Fix failing unit tests Signed-off-by: Suraj Singh --- .../org/opensearch/index/store/Store.java | 3 +- .../RemoteStoreReplicationSource.java | 22 +++--- .../RemoteStoreRefreshListenerTests.java | 2 +- ...overyWithRemoteTranslogOnPrimaryTests.java | 56 ++++++++++++++- .../RemoteSegmentMetadataHandlerTests.java | 25 ++++--- ...teStorePeerRecoverySourceHandlerTests.java | 68 +++++++++++++++++-- 6 files changed, 145 insertions(+), 31 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index 04bfe49ba4e70..921deae41946a 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -845,7 +845,8 @@ private void cleanupFiles(Collection filesToConsiderForCleanup, String r * @param tmpToFileName Map of temporary replication file to actual file name * @param infosBytes bytes[] of SegmentInfos supposed to be sent over by primary excluding segment_N file * @param segmentsGen segment generation number - * @param finalizeConsumer consumer for generated SegmentInfos + * @param finalizeConsumer consumer for action on passed in SegmentInfos + * @param renameConsumer consumer for action on temporary copied over files * @throws IOException Exception while reading store and building segment infos */ public void buildInfosFromBytes( diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index a97cec2c904e2..89bf1870c7288 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -40,9 +40,13 @@ public class RemoteStoreReplicationSource implements SegmentReplicationSource { private static final Logger logger = LogManager.getLogger(RemoteStoreReplicationSource.class); private final IndexShard indexShard; + private final RemoteSegmentStoreDirectory remoteDirectory; public RemoteStoreReplicationSource(IndexShard indexShard) { this.indexShard = indexShard; + FilterDirectory remoteStoreDirectory = (FilterDirectory) indexShard.remoteStore().directory(); + FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); + this.remoteDirectory = (RemoteSegmentStoreDirectory) byteSizeCachingStoreDirectory.getDelegate(); } @Override @@ -51,15 +55,11 @@ public void getCheckpointMetadata( ReplicationCheckpoint checkpoint, ActionListener listener ) { - FilterDirectory remoteStoreDirectory = (FilterDirectory) indexShard.remoteStore().directory(); - FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); - RemoteSegmentStoreDirectory remoteDirectory = (RemoteSegmentStoreDirectory) byteSizeCachingStoreDirectory.getDelegate(); - Map metadataMap; // TODO: Need to figure out a way to pass this information for segment metadata via remote store. final Version version = indexShard.getSegmentInfosSnapshot().get().getCommitLuceneVersion(); try { - RemoteSegmentMetadata mdFile = remoteDirectory.readLatestMetadataFile(); + RemoteSegmentMetadata mdFile = remoteDirectory.init(); // During initial recovery flow, the remote store might not have metadata as primary hasn't uploaded anything yet. if (mdFile == null && indexShard.state().equals(IndexShardState.STARTED) == false) { listener.onResponse(new CheckpointInfoResponse(checkpoint, Collections.emptyMap(), null)); @@ -96,12 +96,12 @@ public void getSegmentFiles( ActionListener listener ) { try { + if (filesToFetch.isEmpty()) { + listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); + return; + } logger.trace("Downloading segments files from remote store {}", filesToFetch); - FilterDirectory remoteStoreDirectory = (FilterDirectory) indexShard.remoteStore().directory(); - FilterDirectory byteSizeCachingStoreDirectory = (FilterDirectory) remoteStoreDirectory.getDelegate(); - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) byteSizeCachingStoreDirectory - .getDelegate(); - RemoteSegmentMetadata remoteSegmentMetadata = remoteSegmentStoreDirectory.init(); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); List downloadedSegments = new ArrayList<>(); if (remoteSegmentMetadata != null) { try { @@ -111,7 +111,7 @@ public void getSegmentFiles( String segmentNFile = null; for (StoreFileMetadata fileMetadata : filesToFetch) { String file = fileMetadata.name(); - storeDirectory.copyFrom(remoteSegmentStoreDirectory, file, file, IOContext.DEFAULT); + storeDirectory.copyFrom(remoteDirectory, file, file, IOContext.DEFAULT); downloadedSegments.add(fileMetadata); if (file.startsWith(IndexFileNames.SEGMENTS)) { assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; diff --git a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java index f13f89c6e067c..66938eec10513 100644 --- a/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java +++ b/server/src/test/java/org/opensearch/index/shard/RemoteStoreRefreshListenerTests.java @@ -413,7 +413,7 @@ private Tuple m when(remoteStore.directory()).thenReturn(remoteStoreFilterDirectory); // Mock indexShard.getOperationPrimaryTerm() - when(shard.getOperationPrimaryTerm()).thenReturn(indexShard.getOperationPrimaryTerm()); + when(shard.getLatestReplicationCheckpoint()).thenReturn(indexShard.getLatestReplicationCheckpoint()); // Mock indexShard.routingEntry().primary() when(shard.routingEntry()).thenReturn(indexShard.routingEntry()); diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index 690c7955ff338..e005ef2674ba8 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -9,24 +9,36 @@ package org.opensearch.index.shard; import org.junit.Assert; +import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.NRTReplicationEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.mapper.MapperService; import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.translog.WriteOnlyTranslogManager; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryTarget; +import org.opensearch.indices.replication.SegmentReplicationSourceFactory; +import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.transport.TransportService; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; @@ -39,6 +51,14 @@ public class ReplicaRecoveryWithRemoteTranslogOnPrimaryTests extends OpenSearchI .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "100ms") .build(); + @Before + public void setup() { + // Todo: Remove feature flag once remote store integration with segrep goes GA + FeatureFlags.initializeFeatureFlags( + Settings.builder().put(FeatureFlags.SEGMENT_REPLICATION_EXPERIMENTAL_SETTING.getKey(), "true").build() + ); + } + public void testStartSequenceForReplicaRecovery() throws Exception { try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { @@ -111,11 +131,14 @@ public IndexShard indexShard() { } public void testNoTranslogHistoryTransferred() throws Exception { - try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { + final Path remoteDir = createTempDir(); + final String indexMapping = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": {} }"; + try (ReplicationGroup shards = createGroup(0, settings, indexMapping, new NRTReplicationEngineFactory(), remoteDir)) { // Step1 - Start primary, index docs, flush, index more docs, check translog in primary as expected shards.startPrimary(); final IndexShard primary = shards.getPrimary(); + logger.info("--> Index docs on primary and flush"); int numDocs = shards.indexDocs(randomIntBetween(10, 100)); shards.flush(); List docIdAndSeqNosAfterFlush = getDocIdAndSeqNos(primary); @@ -123,7 +146,8 @@ public void testNoTranslogHistoryTransferred() throws Exception { assertEquals(numDocs + moreDocs, getTranslog(primary).totalOperations()); // Step 2 - Start replica, recovery happens, check docs recovered till last flush - final IndexShard replica = shards.addReplica(); + logger.info("--> Add replica shard and start"); + final IndexShard replica = shards.addReplica(remoteDir); shards.startAll(); assertEquals(docIdAndSeqNosAfterFlush, getDocIdAndSeqNos(replica)); assertDocCount(replica, numDocs); @@ -140,4 +164,32 @@ public void testNoTranslogHistoryTransferred() throws Exception { shards.assertAllEqual(numDocs + moreDocs); } } + + protected SegmentReplicationTargetService prepareForReplication( + IndexShard primaryShard, + IndexShard target, + TransportService transportService, + IndicesService indicesService, + ClusterService clusterService, + Consumer postGetFilesRunnable + ) { + RecoverySettings recoverySettings = new RecoverySettings( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + SegmentReplicationSourceFactory sourceFactory = new SegmentReplicationSourceFactory( + transportService, + recoverySettings, + clusterService + ); + final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( + threadPool, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + transportService, + sourceFactory, + indicesService, + clusterService + ); + return targetService; + } } diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java index c891623619bde..581605961dfd8 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java @@ -25,6 +25,8 @@ import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardTestCase; import org.opensearch.index.store.Store; +import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.common.ReplicationType; import java.io.IOException; import java.util.HashMap; @@ -38,16 +40,23 @@ public class RemoteSegmentMetadataHandlerTests extends IndexShardTestCase { private IndexShard indexShard; private SegmentInfos segmentInfos; + private ReplicationCheckpoint replicationCheckpoint; + @Before public void setup() throws IOException { remoteSegmentMetadataHandler = new RemoteSegmentMetadataHandler(); - Settings indexSettings = Settings.builder().put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT).build(); + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_VERSION_CREATED, org.opensearch.Version.CURRENT) + .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) + .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) + .build(); indexShard = newStartedShard(false, indexSettings, new NRTReplicationEngineFactory()); try (Store store = indexShard.store()) { segmentInfos = store.readLastCommittedSegmentsInfo(); } + replicationCheckpoint = indexShard.getLatestReplicationCheckpoint(); } @After @@ -61,8 +70,7 @@ public void testReadContentNoSegmentInfos() throws IOException { OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); - indexOutput.writeLong(1234); - indexOutput.writeLong(1234); + replicationCheckpoint.writeTo(indexOutput); indexOutput.writeLong(0); indexOutput.writeBytes(new byte[0], 0); indexOutput.close(); @@ -70,7 +78,7 @@ public void testReadContentNoSegmentInfos() throws IOException { new ByteArrayIndexInput("dummy bytes", BytesReference.toBytes(output.bytes())) ); assertEquals(expectedOutput, metadata.toMapOfStrings()); - assertEquals(1234, metadata.getGeneration()); + assertEquals(replicationCheckpoint.getSegmentsGen(), metadata.getGeneration()); } public void testReadContentWithSegmentInfos() throws IOException { @@ -78,8 +86,7 @@ public void testReadContentWithSegmentInfos() throws IOException { OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); - indexOutput.writeLong(1234); - indexOutput.writeLong(1234); + replicationCheckpoint.writeTo(indexOutput); ByteBuffersIndexOutput segmentInfosOutput = new ByteBuffersIndexOutput(new ByteBuffersDataOutput(), "test", "resource"); segmentInfos.write(segmentInfosOutput); byte[] segmentInfosBytes = segmentInfosOutput.toArrayCopy(); @@ -90,7 +97,7 @@ public void testReadContentWithSegmentInfos() throws IOException { new ByteArrayIndexInput("dummy bytes", BytesReference.toBytes(output.bytes())) ); assertEquals(expectedOutput, metadata.toMapOfStrings()); - assertEquals(1234, metadata.getGeneration()); + assertEquals(replicationCheckpoint.getSegmentsGen(), metadata.getGeneration()); assertArrayEquals(segmentInfosBytes, metadata.getSegmentInfosBytes()); } @@ -115,8 +122,8 @@ public void testWriteContent() throws IOException { new ByteArrayIndexInput("dummy bytes", BytesReference.toBytes(output.bytes())) ); assertEquals(expectedOutput, metadata.toMapOfStrings()); - assertEquals(1234, metadata.getGeneration()); - assertEquals(1234, metadata.getPrimaryTerm()); + assertEquals(replicationCheckpoint.getSegmentsGen(), metadata.getGeneration()); + assertEquals(replicationCheckpoint.getPrimaryTerm(), metadata.getPrimaryTerm()); assertArrayEquals(segmentInfosBytes, metadata.getSegmentInfosBytes()); } diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index 8135d9cd3718e..c9765e62afeed 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -8,14 +8,26 @@ package org.opensearch.indices.recovery; +import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.cluster.service.ClusterService; +import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexSettings; import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.mapper.MapperService; import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.shard.IndexShard; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.SegmentReplicationSourceFactory; +import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.transport.TransportService; + +import java.nio.file.Path; +import java.util.function.Consumer; public class RemoteStorePeerRecoverySourceHandlerTests extends OpenSearchIndexLevelReplicationTestCase { @@ -26,9 +38,20 @@ public class RemoteStorePeerRecoverySourceHandlerTests extends OpenSearchIndexLe .put(IndexSettings.INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.getKey(), "100ms") .build(); + @Before + public void setup() { + // Todo: Remove feature flag once remote store integration with segrep goes GA + FeatureFlags.initializeFeatureFlags( + Settings.builder().put(FeatureFlags.SEGMENT_REPLICATION_EXPERIMENTAL_SETTING.getKey(), "true").build() + ); + } + public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { - try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { + final Path remoteDir = createTempDir(); + final String indexMapping = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": {} }"; + try (ReplicationGroup shards = createGroup(0, settings, indexMapping, new NRTReplicationEngineFactory(), remoteDir)) { + logger.info("--> Start primary shard, index docs and flush"); // Step1 - Start primary, index docs and flush shards.startPrimary(); final IndexShard primary = shards.getPrimary(); @@ -36,11 +59,13 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { shards.flush(); // Step 2 - Start replica for recovery to happen, check both has same number of docs - final IndexShard replica1 = shards.addReplica(); + logger.info("--> Add replica shard"); + final IndexShard replica1 = shards.addReplica(remoteDir); shards.startAll(); assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); // Step 3 - Index more docs, run segment replication, check both have same number of docs + logger.info("--> Add more docs and verify"); int moreDocs = shards.indexDocs(randomIntBetween(10, 100)); primary.refresh("test"); replicateSegments(primary, shards.getReplicas()); @@ -55,13 +80,42 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica1.routingEntry()))); // Step 6 - Start new replica, recovery happens, and check that new replica has all docs - final IndexShard replica2 = shards.addReplica(); + logger.info("--> Add another replica shard"); + final IndexShard replica2 = shards.addReplica(remoteDir); shards.startAll(); shards.assertAllEqual(numDocs + moreDocs); - - // Step 7 - Check retention lease does not exist for the replica shard - assertEquals(1, primary.getRetentionLeases().leases().size()); - assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica2.routingEntry()))); + // + // // Step 7 - Check retention lease does not exist for the replica shard + // assertEquals(1, primary.getRetentionLeases().leases().size()); + // assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica2.routingEntry()))); } } + + protected SegmentReplicationTargetService prepareForReplication( + IndexShard primaryShard, + IndexShard target, + TransportService transportService, + IndicesService indicesService, + ClusterService clusterService, + Consumer postGetFilesRunnable + ) { + RecoverySettings recoverySettings = new RecoverySettings( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + SegmentReplicationSourceFactory sourceFactory = new SegmentReplicationSourceFactory( + transportService, + recoverySettings, + clusterService + ); + final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( + threadPool, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + transportService, + sourceFactory, + indicesService, + clusterService + ); + return targetService; + } } From 7200dcb969a4e1e1dce372e4d92e0cdf5ab516a8 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Sun, 23 Jul 2023 22:17:19 -0700 Subject: [PATCH 07/26] Fix failing UT Signed-off-by: Suraj Singh --- .../index/store/RemoteSegmentStoreDirectoryTests.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 82ab2d4b79b81..9de731bf63dd6 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -700,7 +700,7 @@ public void testUploadMetadataNonEmpty() throws IOException { indexDocs(142364, 5); flushShard(indexShard, true); SegmentInfos segInfos = indexShard.store().readLastCommittedSegmentsInfo(); - long primaryTerm = 12; + long primaryTerm = indexShard.getLatestReplicationCheckpoint().getPrimaryTerm(); String primaryTermLong = RemoteStoreUtils.invertLong(primaryTerm); long generation = segInfos.getGeneration(); String generationLong = RemoteStoreUtils.invertLong(generation); @@ -717,7 +717,7 @@ public void testUploadMetadataNonEmpty() throws IOException { getDummyMetadata("_0", (int) generation) ); when(remoteMetadataDirectory.openInput(latestMetadataFileName, IOContext.DEFAULT)).thenReturn( - createMetadataFileBytes(metadataFilenameContentMapping.get(latestMetadataFileName), generation, primaryTerm) + createMetadataFileBytes(metadataFilenameContentMapping.get(latestMetadataFileName), indexShard.getLatestReplicationCheckpoint()) ); remoteSegmentStoreDirectory.init(); From 8c92bc3c64a911e87f71d63918f0bca57246db30 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Sun, 23 Jul 2023 22:30:57 -0700 Subject: [PATCH 08/26] Fix failing UT Signed-off-by: Suraj Singh --- .../index/store/RemoteSegmentStoreDirectoryTests.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 9de731bf63dd6..32a780484992f 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -728,7 +728,13 @@ public void testUploadMetadataNonEmpty() throws IOException { when(storeDirectory.createOutput(startsWith("metadata__" + primaryTermLong + "__" + generationLong), eq(IOContext.DEFAULT))) .thenReturn(indexOutput); - remoteSegmentStoreDirectory.uploadMetadata(segInfos.files(true), segInfos, storeDirectory, generation, indexShard.getLatestReplicationCheckpoint()); + remoteSegmentStoreDirectory.uploadMetadata( + segInfos.files(true), + segInfos, + storeDirectory, + generation, + indexShard.getLatestReplicationCheckpoint() + ); verify(remoteMetadataDirectory).copyFrom( eq(storeDirectory), From 49cabde9cd727b2e774d81cff284c364ff04b7bf Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Mon, 24 Jul 2023 15:25:53 -0700 Subject: [PATCH 09/26] Address review comments Signed-off-by: Suraj Singh --- .../store/remote/metadata/RemoteSegmentMetadata.java | 8 ++++---- .../indices/replication/RemoteStoreReplicationSource.java | 4 ++-- .../indices/replication/SegmentReplicationTarget.java | 2 +- .../RemoteStorePeerRecoverySourceHandlerTests.java | 8 ++++---- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index befec685754c2..96e52e29a6687 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -39,10 +39,6 @@ public class RemoteSegmentMetadata { private final byte[] segmentInfosBytes; - public ReplicationCheckpoint getReplicationCheckpoint() { - return replicationCheckpoint; - } - private final ReplicationCheckpoint replicationCheckpoint; public RemoteSegmentMetadata( @@ -75,6 +71,10 @@ public long getPrimaryTerm() { return replicationCheckpoint.getPrimaryTerm(); } + public ReplicationCheckpoint getReplicationCheckpoint() { + return replicationCheckpoint; + } + /** * Generate {@code Map} from {@link RemoteSegmentMetadata} * @return {@code Map} diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 89bf1870c7288..4d6ef2795110b 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -59,7 +59,7 @@ public void getCheckpointMetadata( // TODO: Need to figure out a way to pass this information for segment metadata via remote store. final Version version = indexShard.getSegmentInfosSnapshot().get().getCommitLuceneVersion(); try { - RemoteSegmentMetadata mdFile = remoteDirectory.init(); + RemoteSegmentMetadata mdFile = remoteDirectory.readLatestMetadataFile(); // During initial recovery flow, the remote store might not have metadata as primary hasn't uploaded anything yet. if (mdFile == null && indexShard.state().equals(IndexShardState.STARTED) == false) { listener.onResponse(new CheckpointInfoResponse(checkpoint, Collections.emptyMap(), null)); @@ -101,7 +101,7 @@ public void getSegmentFiles( return; } logger.trace("Downloading segments files from remote store {}", filesToFetch); - RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); List downloadedSegments = new ArrayList<>(); if (remoteSegmentMetadata != null) { try { diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index ee4490e94fdb5..334e364040218 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -198,7 +198,7 @@ private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, try { store = store(); store.incRef(); - Map tempFileNames = null; + Map tempFileNames; if (this.indexShard.indexSettings().isRemoteStoreEnabled() == true) { tempFileNames = getSegmentFilesResponse.getFiles() != null ? getSegmentFilesResponse.getFiles() diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index c9765e62afeed..0a3d8b3a9f8fe 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -84,10 +84,10 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { final IndexShard replica2 = shards.addReplica(remoteDir); shards.startAll(); shards.assertAllEqual(numDocs + moreDocs); - // - // // Step 7 - Check retention lease does not exist for the replica shard - // assertEquals(1, primary.getRetentionLeases().leases().size()); - // assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica2.routingEntry()))); + + // Step 7 - Check retention lease does not exist for the replica shard + assertEquals(1, primary.getRetentionLeases().leases().size()); + assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica2.routingEntry()))); } } From 2ce7e6338654acfecda338d795ee74199ca229b2 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 25 Jul 2023 13:38:53 -0700 Subject: [PATCH 10/26] Fix more unit tests Signed-off-by: Suraj Singh --- .../RemoteStoreReplicationSource.java | 6 +- ...overyWithRemoteTranslogOnPrimaryTests.java | 47 +-- .../SegmentReplicationIndexShardTests.java | 279 +---------------- ...licationWithNodeToNodeIndexShardTests.java | 280 ++++++++++++++++++ ...tReplicationWithRemoteIndexShardTests.java | 34 +-- ...teStorePeerRecoverySourceHandlerTests.java | 32 +- .../RemoteStoreReplicationSourceTests.java | 92 ++++-- .../index/shard/IndexShardTestCase.java | 56 ++-- .../indices/recovery/AsyncRecoveryTarget.java | 5 +- 9 files changed, 422 insertions(+), 409 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 4d6ef2795110b..e35100134ab8d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -25,6 +25,7 @@ import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -101,12 +102,13 @@ public void getSegmentFiles( return; } logger.trace("Downloading segments files from remote store {}", filesToFetch); - RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); List downloadedSegments = new ArrayList<>(); if (remoteSegmentMetadata != null) { try { indexShard.store().incRef(); indexShard.remoteStore().incRef(); + Store store = indexShard.store(); final Directory storeDirectory = indexShard.store().directory(); String segmentNFile = null; for (StoreFileMetadata fileMetadata : filesToFetch) { @@ -119,10 +121,10 @@ public void getSegmentFiles( } } storeDirectory.sync(downloadedSegments.stream().map(metadata -> metadata.name()).collect(Collectors.toList())); + logger.trace("Downloaded segments from remote store {}", downloadedSegments); } finally { indexShard.store().decRef(); indexShard.remoteStore().decRef(); - logger.trace("Downloaded segments from remote store {}", downloadedSegments); } } listener.onResponse(new GetSegmentFilesResponse(downloadedSegments)); diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index e005ef2674ba8..982af2caa5d1c 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -25,6 +25,7 @@ import org.opensearch.index.mapper.MapperService; import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.store.Store; import org.opensearch.index.translog.WriteOnlyTranslogManager; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; @@ -60,14 +61,15 @@ public void setup() { } public void testStartSequenceForReplicaRecovery() throws Exception { - try (ReplicationGroup shards = createGroup(0, settings, new NRTReplicationEngineFactory())) { - + final Path remoteDir = createTempDir(); + final String indexMapping = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": {} }"; + try (ReplicationGroup shards = createGroup(0, settings, indexMapping, new NRTReplicationEngineFactory(), remoteDir)) { shards.startPrimary(); final IndexShard primary = shards.getPrimary(); int numDocs = shards.indexDocs(randomIntBetween(10, 100)); shards.flush(); - final IndexShard replica = shards.addReplica(); + final IndexShard replica = shards.addReplica(remoteDir); shards.startAll(); allowShardFailures(); @@ -83,6 +85,14 @@ public void testStartSequenceForReplicaRecovery() throws Exception { int moreDocs = shards.indexDocs(randomIntBetween(20, 100)); shards.flush(); + final ShardRouting replicaRouting2 = newShardRouting( + replicaRouting.shardId(), + replicaRouting.currentNodeId(), + false, + ShardRoutingState.INITIALIZING, + RecoverySource.PeerRecoverySource.INSTANCE + ); + Store remoteStore = createRemoteStore(remoteDir, replicaRouting2, newIndexMetadata); IndexShard newReplicaShard = newShard( newShardRouting( replicaRouting.shardId(), @@ -100,7 +110,7 @@ public void testStartSequenceForReplicaRecovery() throws Exception { replica.getGlobalCheckpointSyncer(), replica.getRetentionLeaseSyncer(), EMPTY_EVENT_LISTENER, - null + remoteStore ); shards.addReplica(newReplicaShard); AtomicBoolean assertDone = new AtomicBoolean(false); @@ -123,7 +133,6 @@ public IndexShard indexShard() { return idxShard; } }); - shards.flush(); replicateSegments(primary, shards.getReplicas()); shards.assertAllEqual(numDocs + moreDocs); @@ -164,32 +173,4 @@ public void testNoTranslogHistoryTransferred() throws Exception { shards.assertAllEqual(numDocs + moreDocs); } } - - protected SegmentReplicationTargetService prepareForReplication( - IndexShard primaryShard, - IndexShard target, - TransportService transportService, - IndicesService indicesService, - ClusterService clusterService, - Consumer postGetFilesRunnable - ) { - RecoverySettings recoverySettings = new RecoverySettings( - Settings.EMPTY, - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); - SegmentReplicationSourceFactory sourceFactory = new SegmentReplicationSourceFactory( - transportService, - recoverySettings, - clusterService - ); - final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( - threadPool, - new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), - transportService, - sourceFactory, - indicesService, - clusterService - ); - return targetService; - } } diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 1a33473f04768..54cbcb9df2e55 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -143,8 +143,10 @@ public void testReplication() throws Exception { final IndexShard replicaShard = shards.getReplicas().get(0); // index and replicate segments to replica. - int numDocs = randomIntBetween(10, 100); + int numDocs = randomIntBetween(10, 20); + logger.info("--> Index {} docs", numDocs); shards.indexDocs(numDocs); + logger.info("--> Refresh"); primaryShard.refresh("test"); flushShard(primaryShard); replicateSegments(primaryShard, List.of(replicaShard)); @@ -203,7 +205,7 @@ public void testSegmentInfosAndReplicationCheckpointTuple() throws Exception { assertEquals(1, primary.getLatestReplicationCheckpoint().compareTo(replica.getLatestReplicationCheckpoint())); // index and copy segments to replica. - int numDocs = randomIntBetween(10, 100); + int numDocs = randomIntBetween(10, 20); shards.indexDocs(numDocs); primary.refresh("test"); replicateSegments(primary, List.of(replica)); @@ -253,7 +255,7 @@ public void testSegmentReplication_With_ReaderClosedConcurrently() throws Except final IndexShard replicaShard = shards.getReplicas().get(0); // Step 1. Ingest numDocs documents & replicate to replica shard - final int numDocs = randomIntBetween(100, 200); + final int numDocs = randomIntBetween(10, 20); logger.info("--> Inserting documents {}", numDocs); for (int i = 0; i < numDocs; i++) { shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); @@ -302,7 +304,7 @@ public void testSegmentReplication_With_EngineClosedConcurrently() throws Except final IndexShard replicaShard = shards.getReplicas().get(0); // Step 1. Ingest numDocs documents - final int numDocs = randomIntBetween(100, 200); + final int numDocs = randomIntBetween(10, 20); logger.info("--> Inserting documents {}", numDocs); for (int i = 0; i < numDocs; i++) { shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); @@ -338,56 +340,6 @@ public void testSegmentReplication_With_EngineClosedConcurrently() throws Except } } - public void testSegmentReplication_Index_Update_Delete() throws Exception { - String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; - try (ReplicationGroup shards = createGroup(2, getIndexSettings(), mappings, new NRTReplicationEngineFactory())) { - shards.startAll(); - final IndexShard primaryShard = shards.getPrimary(); - - final int numDocs = randomIntBetween(100, 200); - for (int i = 0; i < numDocs; i++) { - shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); - } - - assertEqualTranslogOperations(shards, primaryShard); - primaryShard.refresh("Test"); - replicateSegments(primaryShard, shards.getReplicas()); - - shards.assertAllEqual(numDocs); - - for (int i = 0; i < numDocs; i++) { - // randomly update docs. - if (randomBoolean()) { - shards.index( - new IndexRequest(index.getName()).id(String.valueOf(i)).source("{ \"foo\" : \"baz\" }", XContentType.JSON) - ); - } - } - assertEqualTranslogOperations(shards, primaryShard); - primaryShard.refresh("Test"); - replicateSegments(primaryShard, shards.getReplicas()); - shards.assertAllEqual(numDocs); - - final List docs = getDocIdAndSeqNos(primaryShard); - for (IndexShard shard : shards.getReplicas()) { - assertEquals(getDocIdAndSeqNos(shard), docs); - } - for (int i = 0; i < numDocs; i++) { - // randomly delete. - if (randomBoolean()) { - shards.delete(new DeleteRequest(index.getName()).id(String.valueOf(i))); - } - } - assertEqualTranslogOperations(shards, primaryShard); - primaryShard.refresh("Test"); - replicateSegments(primaryShard, shards.getReplicas()); - final List docsAfterDelete = getDocIdAndSeqNos(primaryShard); - for (IndexShard shard : shards.getReplicas()) { - assertEquals(getDocIdAndSeqNos(shard), docsAfterDelete); - } - } - } - public void testIgnoreShardIdle() throws Exception { Settings updatedSettings = Settings.builder() .put(getIndexSettings()) @@ -520,51 +472,6 @@ public void testRejectCheckpointOnShardRoutingPrimary() throws IOException { closeShards(primaryShard); } - public void testPrimaryRelocationWithSegRepFailure() throws Exception { - final IndexShard primarySource = newStartedShard(true, getIndexSettings()); - int totalOps = randomInt(10); - for (int i = 0; i < totalOps; i++) { - indexDoc(primarySource, "_doc", Integer.toString(i)); - } - IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1)); - final IndexShard primaryTarget = newShard( - primarySource.routingEntry().getTargetRelocatingShard(), - getIndexSettings(), - new NRTReplicationEngineFactory() - ); - updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetadata()); - - Function, List> replicatePrimaryFunction = (shardList) -> { - try { - throw new IOException("Expected failure"); - } catch (IOException e) { - throw new RuntimeException(e); - } - }; - Exception e = expectThrows( - Exception.class, - () -> recoverReplica( - primaryTarget, - primarySource, - (primary, sourceNode) -> new RecoveryTarget(primary, sourceNode, new ReplicationListener() { - @Override - public void onDone(ReplicationState state) { - throw new AssertionError("recovery must fail"); - } - - @Override - public void onFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { - assertEquals(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), "Expected failure"); - } - }), - true, - true, - replicatePrimaryFunction - ) - ); - closeShards(primarySource, primaryTarget); - } - // Todo: Remove this test when there is a better mechanism to test a functionality passing in different replication // strategy. public void testLockingBeforeAndAfterRelocated() throws Exception { @@ -675,90 +582,6 @@ public void onFailure(Exception e) { closeShards(shard); } - public void testReplicaClosesWhileReplicating_AfterGetCheckpoint() throws Exception { - try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { - shards.startAll(); - IndexShard primary = shards.getPrimary(); - final IndexShard replica = shards.getReplicas().get(0); - - final int numDocs = shards.indexDocs(randomInt(10)); - primary.refresh("Test"); - - final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); - final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); - SegmentReplicationSource source = new TestReplicationSource() { - @Override - public void getCheckpointMetadata( - long replicationId, - ReplicationCheckpoint checkpoint, - ActionListener listener - ) { - // trigger a cancellation by closing the replica. - targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY); - resolveCheckpointInfoResponseListener(listener, primary); - } - - @Override - public void getSegmentFiles( - long replicationId, - ReplicationCheckpoint checkpoint, - List filesToFetch, - IndexShard indexShard, - ActionListener listener - ) { - Assert.fail("Should not be reached"); - } - }; - when(sourceFactory.get(any())).thenReturn(source); - startReplicationAndAssertCancellation(replica, targetService); - - shards.removeReplica(replica); - closeShards(replica); - } - } - - public void testReplicaClosesWhileReplicating_AfterGetSegmentFiles() throws Exception { - try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { - shards.startAll(); - IndexShard primary = shards.getPrimary(); - final IndexShard replica = shards.getReplicas().get(0); - - final int numDocs = shards.indexDocs(randomInt(10)); - primary.refresh("Test"); - - final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); - final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); - SegmentReplicationSource source = new TestReplicationSource() { - @Override - public void getCheckpointMetadata( - long replicationId, - ReplicationCheckpoint checkpoint, - ActionListener listener - ) { - resolveCheckpointInfoResponseListener(listener, primary); - } - - @Override - public void getSegmentFiles( - long replicationId, - ReplicationCheckpoint checkpoint, - List filesToFetch, - IndexShard indexShard, - ActionListener listener - ) { - // randomly resolve the listener, indicating the source has resolved. - listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); - targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY); - } - }; - when(sourceFactory.get(any())).thenReturn(source); - startReplicationAndAssertCancellation(replica, targetService); - - shards.removeReplica(replica); - closeShards(replica); - } - } - public void testCloseShardDuringFinalize() throws Exception { try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); @@ -774,58 +597,6 @@ public void testCloseShardDuringFinalize() throws Exception { } } - public void testCloseShardWhileGettingCheckpoint() throws Exception { - try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { - shards.startAll(); - IndexShard primary = shards.getPrimary(); - final IndexShard replica = shards.getReplicas().get(0); - - primary.refresh("Test"); - - final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); - final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); - SegmentReplicationSource source = new TestReplicationSource() { - - ActionListener listener; - - @Override - public void getCheckpointMetadata( - long replicationId, - ReplicationCheckpoint checkpoint, - ActionListener listener - ) { - // set the listener, we will only fail it once we cancel the source. - this.listener = listener; - // shard is closing while we are copying files. - targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY); - } - - @Override - public void getSegmentFiles( - long replicationId, - ReplicationCheckpoint checkpoint, - List filesToFetch, - IndexShard indexShard, - ActionListener listener - ) { - Assert.fail("Unreachable"); - } - - @Override - public void cancel() { - // simulate listener resolving, but only after we have issued a cancel from beforeIndexShardClosed . - final RuntimeException exception = new CancellableThreads.ExecutionCancelledException("retryable action was cancelled"); - listener.onFailure(exception); - } - }; - when(sourceFactory.get(any())).thenReturn(source); - startReplicationAndAssertCancellation(replica, targetService); - - shards.removeReplica(replica); - closeShards(replica); - } - } - public void testBeforeIndexShardClosedWhileCopyingFiles() throws Exception { try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); @@ -878,44 +649,6 @@ public void cancel() { } } - public void testPrimaryCancelsExecution() throws Exception { - try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { - shards.startAll(); - IndexShard primary = shards.getPrimary(); - final IndexShard replica = shards.getReplicas().get(0); - - final int numDocs = shards.indexDocs(randomInt(10)); - primary.refresh("Test"); - - final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); - final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); - SegmentReplicationSource source = new TestReplicationSource() { - @Override - public void getCheckpointMetadata( - long replicationId, - ReplicationCheckpoint checkpoint, - ActionListener listener - ) { - listener.onFailure(new CancellableThreads.ExecutionCancelledException("Cancelled")); - } - - @Override - public void getSegmentFiles( - long replicationId, - ReplicationCheckpoint checkpoint, - List filesToFetch, - IndexShard indexShard, - ActionListener listener - ) {} - }; - when(sourceFactory.get(any())).thenReturn(source); - startReplicationAndAssertCancellation(replica, targetService); - - shards.removeReplica(replica); - closeShards(replica); - } - } - protected SegmentReplicationTargetService newTargetService(SegmentReplicationSourceFactory sourceFactory) { return new SegmentReplicationTargetService( threadPool, diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java index dc39b0c27ae1e..50189d0808cd2 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java @@ -10,14 +10,17 @@ import org.apache.lucene.index.SegmentInfos; import org.junit.Assert; +import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.flush.FlushRequest; +import org.opensearch.action.delete.DeleteRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.common.concurrent.GatedCloseable; import org.opensearch.common.lease.Releasable; import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.CancellableThreads; import org.opensearch.common.xcontent.XContentType; import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.InternalEngine; @@ -28,6 +31,7 @@ import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.replication.CheckpointInfoResponse; import org.opensearch.indices.replication.GetSegmentFilesResponse; import org.opensearch.indices.replication.SegmentReplicationSource; @@ -35,6 +39,9 @@ import org.opensearch.indices.replication.SegmentReplicationTarget; import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; +import org.opensearch.indices.replication.common.ReplicationFailedException; +import org.opensearch.indices.replication.common.ReplicationListener; +import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -58,6 +65,180 @@ public class SegmentReplicationWithNodeToNodeIndexShardTests extends SegmentReplicationIndexShardTests { + public void testReplicaClosesWhileReplicating_AfterGetSegmentFiles() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + + final int numDocs = shards.indexDocs(randomInt(10)); + primary.refresh("Test"); + + final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); + final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); + SegmentReplicationSource source = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) { + resolveCheckpointInfoResponseListener(listener, primary); + } + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + ActionListener listener + ) { + // randomly resolve the listener, indicating the source has resolved. + listener.onResponse(new GetSegmentFilesResponse(Collections.emptyList())); + targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY); + } + }; + when(sourceFactory.get(any())).thenReturn(source); + startReplicationAndAssertCancellation(replica, targetService); + + shards.removeReplica(replica); + closeShards(replica); + } + } + + public void testReplicaClosesWhileReplicating_AfterGetCheckpoint() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + + final int numDocs = shards.indexDocs(randomInt(10)); + primary.refresh("Test"); + + final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); + final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); + SegmentReplicationSource source = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) { + // trigger a cancellation by closing the replica. + targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY); + resolveCheckpointInfoResponseListener(listener, primary); + } + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + ActionListener listener + ) { + Assert.fail("Should not be reached"); + } + }; + when(sourceFactory.get(any())).thenReturn(source); + startReplicationAndAssertCancellation(replica, targetService); + + shards.removeReplica(replica); + closeShards(replica); + } + } + + public void testCloseShardWhileGettingCheckpoint() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + + primary.refresh("Test"); + + final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); + final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); + SegmentReplicationSource source = new TestReplicationSource() { + + ActionListener listener; + + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) { + // set the listener, we will only fail it once we cancel the source. + this.listener = listener; + // shard is closing while we are copying files. + targetService.beforeIndexShardClosed(replica.shardId, replica, Settings.EMPTY); + } + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + ActionListener listener + ) { + Assert.fail("Unreachable"); + } + + @Override + public void cancel() { + // simulate listener resolving, but only after we have issued a cancel from beforeIndexShardClosed . + final RuntimeException exception = new CancellableThreads.ExecutionCancelledException("retryable action was cancelled"); + listener.onFailure(exception); + } + }; + when(sourceFactory.get(any())).thenReturn(source); + startReplicationAndAssertCancellation(replica, targetService); + + shards.removeReplica(replica); + closeShards(replica); + } + } + + public void testPrimaryCancelsExecution() throws Exception { + try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { + shards.startAll(); + IndexShard primary = shards.getPrimary(); + final IndexShard replica = shards.getReplicas().get(0); + + final int numDocs = shards.indexDocs(randomInt(10)); + primary.refresh("Test"); + + final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); + final SegmentReplicationTargetService targetService = newTargetService(sourceFactory); + SegmentReplicationSource source = new TestReplicationSource() { + @Override + public void getCheckpointMetadata( + long replicationId, + ReplicationCheckpoint checkpoint, + ActionListener listener + ) { + listener.onFailure(new CancellableThreads.ExecutionCancelledException("Cancelled")); + } + + @Override + public void getSegmentFiles( + long replicationId, + ReplicationCheckpoint checkpoint, + List filesToFetch, + IndexShard indexShard, + ActionListener listener + ) {} + }; + when(sourceFactory.get(any())).thenReturn(source); + startReplicationAndAssertCancellation(replica, targetService); + + shards.removeReplica(replica); + closeShards(replica); + } + } + public void testReplicaPromotedWhileReplicating() throws Exception { try (ReplicationGroup shards = createGroup(1, getIndexSettings(), new NRTReplicationEngineFactory())) { shards.startAll(); @@ -466,4 +647,103 @@ public void testReplicaRestarts() throws Exception { } } + // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store + public void testPrimaryRelocationWithSegRepFailure() throws Exception { + final IndexShard primarySource = newStartedShard(true, getIndexSettings()); + int totalOps = randomInt(10); + for (int i = 0; i < totalOps; i++) { + indexDoc(primarySource, "_doc", Integer.toString(i)); + } + IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1)); + final IndexShard primaryTarget = newShard( + primarySource.routingEntry().getTargetRelocatingShard(), + getIndexSettings(), + new NRTReplicationEngineFactory() + ); + updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetadata()); + + Function, List> replicatePrimaryFunction = (shardList) -> { + try { + throw new IOException("Expected failure"); + } catch (IOException e) { + throw new RuntimeException(e); + } + }; + Exception e = expectThrows( + Exception.class, + () -> recoverReplica( + primaryTarget, + primarySource, + (primary, sourceNode) -> new RecoveryTarget(primary, sourceNode, new ReplicationListener() { + @Override + public void onDone(ReplicationState state) { + throw new AssertionError("recovery must fail"); + } + + @Override + public void onFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + assertEquals(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), "Expected failure"); + } + }), + true, + true, + replicatePrimaryFunction + ) + ); + closeShards(primarySource, primaryTarget); + } + + // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store + public void testSegmentReplication_Index_Update_Delete() throws Exception { + String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; + try (ReplicationGroup shards = createGroup(2, getIndexSettings(), mappings, new NRTReplicationEngineFactory())) { + shards.startAll(); + final IndexShard primaryShard = shards.getPrimary(); + + final int numDocs = randomIntBetween(100, 200); + for (int i = 0; i < numDocs; i++) { + shards.index(new IndexRequest(index.getName()).id(String.valueOf(i)).source("{\"foo\": \"bar\"}", XContentType.JSON)); + } + + assertEqualTranslogOperations(shards, primaryShard); + primaryShard.refresh("Test"); + replicateSegments(primaryShard, shards.getReplicas()); + + shards.assertAllEqual(numDocs); + + for (int i = 0; i < numDocs; i++) { + // randomly update docs. + if (randomBoolean()) { + shards.index( + new IndexRequest(index.getName()).id(String.valueOf(i)).source("{ \"foo\" : \"baz\" }", XContentType.JSON) + ); + } + } + assertEqualTranslogOperations(shards, primaryShard); + primaryShard.refresh("Test"); + replicateSegments(primaryShard, shards.getReplicas()); + shards.assertAllEqual(numDocs); + + final List docs = getDocIdAndSeqNos(primaryShard); + for (IndexShard shard : shards.getReplicas()) { + assertEquals(getDocIdAndSeqNos(shard), docs); + } + for (int i = 0; i < numDocs; i++) { + // randomly delete. + if (randomBoolean()) { + shards.delete(new DeleteRequest(index.getName()).id(String.valueOf(i))); + } + } + assertEqualTranslogOperations(shards, primaryShard); + primaryShard.refresh("Test"); + replicateSegments(primaryShard, shards.getReplicas()); + final List docsAfterDelete = getDocIdAndSeqNos(primaryShard); + for (IndexShard shard : shards.getReplicas()) { + assertEquals(getDocIdAndSeqNos(shard), docsAfterDelete); + } + } + } + + + } diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java index b67e7b34b02a5..0cfc32b7963a8 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java @@ -40,20 +40,9 @@ public class SegmentReplicationWithRemoteIndexShardTests extends SegmentReplicat .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, REPOSITORY_NAME) .put(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY, REPOSITORY_NAME) .build(); - TransportService transportService; - IndicesService indicesService; - RecoverySettings recoverySettings; - SegmentReplicationSourceFactory sourceFactory; @Before public void setup() { - recoverySettings = new RecoverySettings( - Settings.EMPTY, - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); - transportService = mock(TransportService.class); - indicesService = mock(IndicesService.class); - sourceFactory = new SegmentReplicationSourceFactory(transportService, recoverySettings, clusterService); // Todo: Remove feature flag once remote store integration with segrep goes GA FeatureFlags.initializeFeatureFlags( Settings.builder().put(FeatureFlags.SEGMENT_REPLICATION_EXPERIMENTAL_SETTING.getKey(), "true").build() @@ -68,25 +57,6 @@ protected ReplicationGroup getReplicationGroup(int numberOfReplicas) throws IOEx return createGroup(numberOfReplicas, settings, indexMapping, new NRTReplicationEngineFactory(), createTempDir()); } - protected SegmentReplicationTargetService prepareForReplication( - IndexShard primaryShard, - IndexShard target, - TransportService transportService, - IndicesService indicesService, - ClusterService clusterService, - Consumer postGetFilesRunnable - ) { - final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( - threadPool, - new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), - transportService, - sourceFactory, - indicesService, - clusterService - ); - return targetService; - } - public void testNRTReplicaWithRemoteStorePromotedAsPrimaryRefreshRefresh() throws Exception { testNRTReplicaWithRemoteStorePromotedAsPrimary(false, false); } @@ -146,6 +116,8 @@ public void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlushF assertTrue(nextPrimary.translogStats().estimatedNumberOfOperations() >= additonalDocs); assertTrue(nextPrimary.translogStats().getUncommittedOperations() >= additonalDocs); + int prevOperationCount = nextPrimary.translogStats().estimatedNumberOfOperations(); + // promote the replica shards.promoteReplicaToPrimary(nextPrimary).get(); @@ -158,7 +130,7 @@ public void testNRTReplicaWithRemoteStorePromotedAsPrimary(boolean performFlushF // As we are downloading segments from remote segment store on failover, there should not be // any operations replayed from translog - assertEquals(0, nextPrimary.translogStats().estimatedNumberOfOperations()); + assertEquals(prevOperationCount, nextPrimary.translogStats().estimatedNumberOfOperations()); // refresh and push segments to our other replica. nextPrimary.refresh("test"); diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index 0a3d8b3a9f8fe..cb9ed219034dd 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -55,7 +55,7 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { // Step1 - Start primary, index docs and flush shards.startPrimary(); final IndexShard primary = shards.getPrimary(); - int numDocs = shards.indexDocs(randomIntBetween(10, 100)); + int numDocs = shards.indexDocs(randomIntBetween(10, 20)); shards.flush(); // Step 2 - Start replica for recovery to happen, check both has same number of docs @@ -66,7 +66,7 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { // Step 3 - Index more docs, run segment replication, check both have same number of docs logger.info("--> Add more docs and verify"); - int moreDocs = shards.indexDocs(randomIntBetween(10, 100)); + int moreDocs = shards.indexDocs(randomIntBetween(10, 20)); primary.refresh("test"); replicateSegments(primary, shards.getReplicas()); assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); @@ -90,32 +90,4 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica2.routingEntry()))); } } - - protected SegmentReplicationTargetService prepareForReplication( - IndexShard primaryShard, - IndexShard target, - TransportService transportService, - IndicesService indicesService, - ClusterService clusterService, - Consumer postGetFilesRunnable - ) { - RecoverySettings recoverySettings = new RecoverySettings( - Settings.EMPTY, - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); - SegmentReplicationSourceFactory sourceFactory = new SegmentReplicationSourceFactory( - transportService, - recoverySettings, - clusterService - ); - final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( - threadPool, - new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), - transportService, - sourceFactory, - indicesService, - clusterService - ); - return targetService; - } } diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index 208ea596bdb67..16c45b224705d 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -8,8 +8,10 @@ package org.opensearch.indices.replication; -import org.apache.lucene.codecs.Codec; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; +import org.apache.lucene.util.Version; import org.mockito.Mockito; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; @@ -21,22 +23,26 @@ import org.opensearch.index.shard.RemoteStoreRefreshListenerTests; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.Store; +import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.ReplicationType; import java.io.IOException; +import java.nio.file.FileAlreadyExistsException; import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class RemoteStoreReplicationSourceTests extends OpenSearchIndexLevelReplicationTestCase { - - private static final long PRIMARY_TERM = 1L; - private static final long SEGMENTS_GEN = 2L; - private static final long VERSION = 4L; private static final long REPLICATION_ID = 123L; private RemoteStoreReplicationSource replicationSource; private IndexShard indexShard; @@ -45,6 +51,8 @@ public class RemoteStoreReplicationSourceTests extends OpenSearchIndexLevelRepli private Store remoteStore; + private Store localStore; + private final Settings settings = Settings.builder() .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, "my-repo") @@ -57,18 +65,18 @@ public void setUp() throws Exception { super.setUp(); indexShard = newStartedShard(true, settings, new InternalEngineFactory()); - indexDoc(indexShard, "_doc", "1"); indexDoc(indexShard, "_doc", "2"); indexShard.refresh("test"); // mock shard mockShard = mock(IndexShard.class); - Store store = mock(Store.class); - when(mockShard.store()).thenReturn(store); - when(store.directory()).thenReturn(indexShard.store().directory()); + localStore = mock(Store.class); + when(mockShard.store()).thenReturn(localStore); + when(localStore.directory()).thenReturn(indexShard.store().directory()); remoteStore = mock(Store.class); when(mockShard.remoteStore()).thenReturn(remoteStore); + when(mockShard.state()).thenReturn(indexShard.state()); RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()).getDelegate()) .getDelegate(); @@ -109,13 +117,15 @@ public void testGetCheckpointMetadataFailure() { } public void testGetCheckpointMetadataEmpty() throws ExecutionException, InterruptedException, IOException { + SegmentInfos segmentInfos = indexShard.getSegmentInfosSnapshot().get(); when(mockShard.getSegmentInfosSnapshot()).thenReturn(indexShard.getSegmentInfosSnapshot()); final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( indexShard.shardId(), - PRIMARY_TERM, - SEGMENTS_GEN, - VERSION, - Codec.getDefault().getName() + indexShard.getOperationPrimaryTerm(), + segmentInfos.getGeneration(), + segmentInfos.getVersion(), + indexShard.store().getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), + indexShard.getDefaultCodecName() ); IndexShard emptyIndexShard = null; try { @@ -134,6 +144,7 @@ public void testGetCheckpointMetadataEmpty() throws ExecutionException, Interrup final PlainActionFuture res = PlainActionFuture.newFuture(); when(mockShard.state()).thenReturn(IndexShardState.RECOVERING); + replicationSource = new RemoteStoreReplicationSource(mockShard); // Recovering shard should just do a noop and return empty metadata map. replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); CheckpointInfoResponse response = res.get(); @@ -151,30 +162,69 @@ public void testGetCheckpointMetadataEmpty() throws ExecutionException, Interrup } } - public void testGetSegmentFiles() throws ExecutionException, InterruptedException { + public void testGetSegmentFiles() throws ExecutionException, InterruptedException, IOException { + SegmentInfos segmentInfos = indexShard.getSegmentInfosSnapshot().get(); final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( indexShard.shardId(), - PRIMARY_TERM, - SEGMENTS_GEN, - VERSION, - Codec.getDefault().getName() + indexShard.getOperationPrimaryTerm(), + segmentInfos.getGeneration(), + segmentInfos.getVersion(), + indexShard.store().getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), + indexShard.getDefaultCodecName() ); - + List filesToFetch = indexShard.getSegmentMetadataMap().values().stream().collect(Collectors.toList()); + when(mockShard.store()).thenReturn(localStore); + Directory directory = mock(Directory.class); + when(localStore.directory()).thenReturn(directory); + doNothing().when(directory).copyFrom(any(), any(), any(), any()); + doNothing().when(directory).sync(any()); final PlainActionFuture res = PlainActionFuture.newFuture(); - replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), indexShard, res); + replicationSource = new RemoteStoreReplicationSource(mockShard); + replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, filesToFetch, indexShard, res); GetSegmentFilesResponse response = res.get(); assert (response.files.isEmpty()); assertEquals("RemoteStoreReplicationSource", replicationSource.getDescription()); + } + public void testGetSegmentFilesFileAlreadyExists() throws IOException, ExecutionException, InterruptedException { + SegmentInfos segmentInfos = indexShard.getSegmentInfosSnapshot().get(); + final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( + indexShard.shardId(), + indexShard.getOperationPrimaryTerm(), + segmentInfos.getGeneration(), + segmentInfos.getVersion(), + indexShard.store().getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), + indexShard.getDefaultCodecName() + ); + List filesToFetch = indexShard.getSegmentMetadataMap().values().stream().collect(Collectors.toList()); + CountDownLatch latch = new CountDownLatch(1); + try { + final PlainActionFuture res = PlainActionFuture.newFuture(); + replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, filesToFetch, indexShard, res); + res.get(); + } catch (Exception ex) { + latch.countDown(); + assertTrue (ex.getCause() instanceof FileAlreadyExistsException); + } + latch.await(); } public void testGetSegmentFilesFailure() throws IOException { final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); + StoreFileMetadata testMetadata = new StoreFileMetadata("testFile", 1L, "checksum", Version.LATEST); Mockito.doThrow(new RuntimeException("testing")).when(mockShard).store(); assertThrows(ExecutionException.class, () -> { final PlainActionFuture res = PlainActionFuture.newFuture(); - replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), mockShard, res); + replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, List.of(testMetadata), mockShard, res); res.get(10, TimeUnit.SECONDS); }); } + + public void testGetSegmentFilesReturnEmptyResponse() throws ExecutionException, InterruptedException { + final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); + final PlainActionFuture res = PlainActionFuture.newFuture(); + replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), mockShard, res); + GetSegmentFilesResponse response = res.get(); + assert (response.files.isEmpty()); + } } diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 7a469819116d6..7b497c7177ae5 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -1326,6 +1326,21 @@ public static Engine.Warmer createTestWarmer(IndexSettings indexSettings) { }; } + private SegmentReplicationTargetService getSegmentReplicationTargetService(TransportService transportService, + IndicesService indicesService, + ClusterService clusterService, + SegmentReplicationSourceFactory sourceFactory + ) { + return new SegmentReplicationTargetService( + threadPool, + new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), + transportService, + sourceFactory, + indicesService, + clusterService + ); + } + /** * Segment Replication specific test method - Creates a {@link SegmentReplicationTargetService} to perform replications that has * been configured to return the given primaryShard's current segments. In order to do so, it mimics the replication @@ -1341,7 +1356,7 @@ public static Engine.Warmer createTestWarmer(IndexSettings indexSettings) { * which are desired right after files are copied. e.g. To work with temp files * @return Returns SegmentReplicationTargetService */ - protected SegmentReplicationTargetService prepareForReplication( + private SegmentReplicationTargetService prepareForReplication( IndexShard primaryShard, IndexShard target, TransportService transportService, @@ -1349,23 +1364,28 @@ protected SegmentReplicationTargetService prepareForReplication( ClusterService clusterService, Consumer postGetFilesRunnable ) { - final SegmentReplicationSourceFactory sourceFactory = mock(SegmentReplicationSourceFactory.class); - final SegmentReplicationTargetService targetService = new SegmentReplicationTargetService( - threadPool, - new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), - transportService, - sourceFactory, - indicesService, - clusterService - ); - final SegmentReplicationSource replicationSource = getSegmentReplicationSource( - primaryShard, - (repId) -> targetService.get(repId), - postGetFilesRunnable - ); - when(sourceFactory.get(any())).thenReturn(replicationSource); - // This is needed for force segment sync call. Remote store uses a different recovery mechanism - when(indicesService.getShardOrNull(any())).thenReturn(target); + + SegmentReplicationSourceFactory sourceFactory = null; + SegmentReplicationTargetService targetService; + if (primaryShard.indexSettings.isRemoteStoreEnabled()) { + RecoverySettings recoverySettings = new RecoverySettings( + Settings.EMPTY, + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) + ); + sourceFactory = new SegmentReplicationSourceFactory(transportService, recoverySettings, clusterService); + targetService = getSegmentReplicationTargetService(transportService, indicesService, clusterService, sourceFactory); + } else { + sourceFactory = mock(SegmentReplicationSourceFactory.class); + targetService = getSegmentReplicationTargetService(transportService, indicesService, clusterService, sourceFactory); + final SegmentReplicationSource replicationSource = getSegmentReplicationSource( + primaryShard, + (repId) -> targetService.get(repId), + postGetFilesRunnable + ); + when(sourceFactory.get(any())).thenReturn(replicationSource); + // This is needed for force segment sync call. Remote store uses a different recovery mechanism + when(indicesService.getShardOrNull(any())).thenReturn(target); + } return targetService; } diff --git a/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java b/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java index b3398fb752be5..6f3b0ec622da1 100644 --- a/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java +++ b/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java @@ -90,7 +90,10 @@ public void prepareForTranslogOperations(int totalTranslogOps, ActionListener Date: Tue, 25 Jul 2023 18:51:12 -0700 Subject: [PATCH 11/26] Improve RemoteStoreReplicationSourceTests, remove unnecessary mocks and use actual failures for failure/exception use cases Signed-off-by: Suraj Singh --- .../RemoteStoreReplicationSource.java | 1 - .../RemoteStoreReplicationSourceTests.java | 196 ++++++------------ 2 files changed, 66 insertions(+), 131 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index e35100134ab8d..614d9e5367c23 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -108,7 +108,6 @@ public void getSegmentFiles( try { indexShard.store().incRef(); indexShard.remoteStore().incRef(); - Store store = indexShard.store(); final Directory storeDirectory = indexShard.store().directory(); String segmentNFile = null; for (StoreFileMetadata fileMetadata : filesToFetch) { diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index 16c45b224705d..921e83c46a96e 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -9,7 +9,6 @@ package org.opensearch.indices.replication; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.util.Version; import org.mockito.Mockito; @@ -17,6 +16,7 @@ import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; import org.opensearch.index.engine.InternalEngineFactory; +import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.shard.IndexShardState; @@ -31,28 +31,19 @@ import java.nio.file.FileAlreadyExistsException; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class RemoteStoreReplicationSourceTests extends OpenSearchIndexLevelReplicationTestCase { private static final long REPLICATION_ID = 123L; private RemoteStoreReplicationSource replicationSource; - private IndexShard indexShard; - - private IndexShard mockShard; - - private Store remoteStore; - - private Store localStore; + private IndexShard primaryShard; + private IndexShard replicaShard; private final Settings settings = Settings.builder() .put(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, true) .put(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY, "my-repo") @@ -63,41 +54,27 @@ public class RemoteStoreReplicationSourceTests extends OpenSearchIndexLevelRepli @Override public void setUp() throws Exception { super.setUp(); - - indexShard = newStartedShard(true, settings, new InternalEngineFactory()); - indexDoc(indexShard, "_doc", "1"); - indexDoc(indexShard, "_doc", "2"); - indexShard.refresh("test"); - - // mock shard - mockShard = mock(IndexShard.class); - localStore = mock(Store.class); - when(mockShard.store()).thenReturn(localStore); - when(localStore.directory()).thenReturn(indexShard.store().directory()); - remoteStore = mock(Store.class); - when(mockShard.remoteStore()).thenReturn(remoteStore); - when(mockShard.state()).thenReturn(indexShard.state()); - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = - (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()).getDelegate()) - .getDelegate(); - FilterDirectory remoteStoreFilterDirectory = new RemoteStoreRefreshListenerTests.TestFilterDirectory( - new RemoteStoreRefreshListenerTests.TestFilterDirectory(remoteSegmentStoreDirectory) + primaryShard = newStartedShard(true, settings, new InternalEngineFactory()); + indexDoc(primaryShard, "_doc", "1"); + indexDoc(primaryShard, "_doc", "2"); + primaryShard.refresh("test"); + replicaShard = newStartedShard( + false, + settings, + new NRTReplicationEngineFactory() ); - when(remoteStore.directory()).thenReturn(remoteStoreFilterDirectory); - replicationSource = new RemoteStoreReplicationSource(mockShard); } @Override public void tearDown() throws Exception { - closeShards(indexShard); + closeShards(primaryShard, replicaShard); super.tearDown(); } public void testGetCheckpointMetadata() throws ExecutionException, InterruptedException { - when(mockShard.getSegmentInfosSnapshot()).thenReturn(indexShard.getSegmentInfosSnapshot()); - final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); - + final ReplicationCheckpoint checkpoint = primaryShard.getLatestReplicationCheckpoint(); final PlainActionFuture res = PlainActionFuture.newFuture(); + replicationSource = new RemoteStoreReplicationSource(primaryShard); replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); CheckpointInfoResponse response = res.get(); assert (response.getCheckpoint().equals(checkpoint)); @@ -105,102 +82,37 @@ public void testGetCheckpointMetadata() throws ExecutionException, InterruptedEx } public void testGetCheckpointMetadataFailure() { - final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); - + IndexShard mockShard = mock(IndexShard.class); + final ReplicationCheckpoint checkpoint = primaryShard.getLatestReplicationCheckpoint(); when(mockShard.getSegmentInfosSnapshot()).thenThrow(new RuntimeException("test")); - assertThrows(RuntimeException.class, () -> { + replicationSource = new RemoteStoreReplicationSource(mockShard); final PlainActionFuture res = PlainActionFuture.newFuture(); replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); res.get(); }); } - public void testGetCheckpointMetadataEmpty() throws ExecutionException, InterruptedException, IOException { - SegmentInfos segmentInfos = indexShard.getSegmentInfosSnapshot().get(); - when(mockShard.getSegmentInfosSnapshot()).thenReturn(indexShard.getSegmentInfosSnapshot()); - final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( - indexShard.shardId(), - indexShard.getOperationPrimaryTerm(), - segmentInfos.getGeneration(), - segmentInfos.getVersion(), - indexShard.store().getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), - indexShard.getDefaultCodecName() - ); - IndexShard emptyIndexShard = null; - try { - emptyIndexShard = newStartedShard( - true, - settings, - new InternalEngineFactory() - ); - RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = - (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) emptyIndexShard.remoteStore().directory()).getDelegate()) - .getDelegate(); - FilterDirectory remoteStoreFilterDirectory = new RemoteStoreRefreshListenerTests.TestFilterDirectory( - new RemoteStoreRefreshListenerTests.TestFilterDirectory(remoteSegmentStoreDirectory) - ); - when(remoteStore.directory()).thenReturn(remoteStoreFilterDirectory); - - final PlainActionFuture res = PlainActionFuture.newFuture(); - when(mockShard.state()).thenReturn(IndexShardState.RECOVERING); - replicationSource = new RemoteStoreReplicationSource(mockShard); - // Recovering shard should just do a noop and return empty metadata map. - replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); - CheckpointInfoResponse response = res.get(); - assert (response.getCheckpoint().equals(checkpoint)); - assert (response.getMetadataMap().isEmpty()); - - when(mockShard.state()).thenReturn(IndexShardState.STARTED); - // Started shard should fail with assertion error. - expectThrows(AssertionError.class, () -> { - final PlainActionFuture res2 = PlainActionFuture.newFuture(); - replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res2); - }); - } finally { - closeShards(emptyIndexShard); - } - } - public void testGetSegmentFiles() throws ExecutionException, InterruptedException, IOException { - SegmentInfos segmentInfos = indexShard.getSegmentInfosSnapshot().get(); - final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( - indexShard.shardId(), - indexShard.getOperationPrimaryTerm(), - segmentInfos.getGeneration(), - segmentInfos.getVersion(), - indexShard.store().getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), - indexShard.getDefaultCodecName() - ); - List filesToFetch = indexShard.getSegmentMetadataMap().values().stream().collect(Collectors.toList()); - when(mockShard.store()).thenReturn(localStore); - Directory directory = mock(Directory.class); - when(localStore.directory()).thenReturn(directory); - doNothing().when(directory).copyFrom(any(), any(), any(), any()); - doNothing().when(directory).sync(any()); + final ReplicationCheckpoint checkpoint = primaryShard.getLatestReplicationCheckpoint(); + List filesToFetch = primaryShard.getSegmentMetadataMap().values().stream().collect(Collectors.toList()); final PlainActionFuture res = PlainActionFuture.newFuture(); - replicationSource = new RemoteStoreReplicationSource(mockShard); - replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, filesToFetch, indexShard, res); + replicationSource = new RemoteStoreReplicationSource(primaryShard); + replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, filesToFetch, replicaShard, res); GetSegmentFilesResponse response = res.get(); - assert (response.files.isEmpty()); - assertEquals("RemoteStoreReplicationSource", replicationSource.getDescription()); + assertEquals(response.files.size(), filesToFetch.size()); + assertTrue(response.files.containsAll(filesToFetch)); + closeShards(replicaShard); } - public void testGetSegmentFilesFileAlreadyExists() throws IOException, ExecutionException, InterruptedException { - SegmentInfos segmentInfos = indexShard.getSegmentInfosSnapshot().get(); - final ReplicationCheckpoint checkpoint = new ReplicationCheckpoint( - indexShard.shardId(), - indexShard.getOperationPrimaryTerm(), - segmentInfos.getGeneration(), - segmentInfos.getVersion(), - indexShard.store().getSegmentMetadataMap(segmentInfos).values().stream().mapToLong(StoreFileMetadata::length).sum(), - indexShard.getDefaultCodecName() - ); - List filesToFetch = indexShard.getSegmentMetadataMap().values().stream().collect(Collectors.toList()); + public void testGetSegmentFilesFileAlreadyExists() throws IOException, InterruptedException { + final ReplicationCheckpoint checkpoint = primaryShard.getLatestReplicationCheckpoint(); + List filesToFetch = primaryShard.getSegmentMetadataMap().values().stream().collect(Collectors.toList()); CountDownLatch latch = new CountDownLatch(1); try { final PlainActionFuture res = PlainActionFuture.newFuture(); - replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, filesToFetch, indexShard, res); + replicationSource = new RemoteStoreReplicationSource(primaryShard); + replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, filesToFetch, primaryShard, res); res.get(); } catch (Exception ex) { latch.countDown(); @@ -209,22 +121,46 @@ public void testGetSegmentFilesFileAlreadyExists() throws IOException, Execution latch.await(); } - public void testGetSegmentFilesFailure() throws IOException { - final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); - StoreFileMetadata testMetadata = new StoreFileMetadata("testFile", 1L, "checksum", Version.LATEST); - Mockito.doThrow(new RuntimeException("testing")).when(mockShard).store(); - assertThrows(ExecutionException.class, () -> { - final PlainActionFuture res = PlainActionFuture.newFuture(); - replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, List.of(testMetadata), mockShard, res); - res.get(10, TimeUnit.SECONDS); - }); - } - public void testGetSegmentFilesReturnEmptyResponse() throws ExecutionException, InterruptedException { - final ReplicationCheckpoint checkpoint = indexShard.getLatestReplicationCheckpoint(); + final ReplicationCheckpoint checkpoint = primaryShard.getLatestReplicationCheckpoint(); final PlainActionFuture res = PlainActionFuture.newFuture(); - replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), mockShard, res); + replicationSource = new RemoteStoreReplicationSource(primaryShard); + replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, Collections.emptyList(), primaryShard, res); GetSegmentFilesResponse response = res.get(); assert (response.files.isEmpty()); } + + public void testGetCheckpointMetadataEmpty() throws ExecutionException, InterruptedException, IOException { + IndexShard mockShard = mock(IndexShard.class); + // Build mockShard to return replicaShard directory so that empty metadata file is returned. + buildIndexShardBehavior(mockShard, replicaShard); + replicationSource = new RemoteStoreReplicationSource(mockShard); + + // Mock replica shard state to RECOVERING so that getCheckpointInfo return empty map + final ReplicationCheckpoint checkpoint = replicaShard.getLatestReplicationCheckpoint(); + final PlainActionFuture res = PlainActionFuture.newFuture(); + when(mockShard.state()).thenReturn(IndexShardState.RECOVERING); + replicationSource = new RemoteStoreReplicationSource(mockShard); + // Recovering shard should just do a noop and return empty metadata map. + replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); + CheckpointInfoResponse response = res.get(); + assert (response.getCheckpoint().equals(checkpoint)); + assert (response.getMetadataMap().isEmpty()); + + // Started shard should fail with assertion error. + when(mockShard.state()).thenReturn(IndexShardState.STARTED); + expectThrows(AssertionError.class, () -> { + final PlainActionFuture res2 = PlainActionFuture.newFuture(); + replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res2); + }); + } + + private void buildIndexShardBehavior(IndexShard mockShard, IndexShard indexShard) { + when(mockShard.getSegmentInfosSnapshot()).thenReturn(indexShard.getSegmentInfosSnapshot()); + Store remoteStore = mock(Store.class); + when(mockShard.remoteStore()).thenReturn(remoteStore); + RemoteSegmentStoreDirectory remoteSegmentStoreDirectory = (RemoteSegmentStoreDirectory) ((FilterDirectory) ((FilterDirectory) indexShard.remoteStore().directory()).getDelegate()).getDelegate(); + FilterDirectory remoteStoreFilterDirectory = new RemoteStoreRefreshListenerTests.TestFilterDirectory(new RemoteStoreRefreshListenerTests.TestFilterDirectory(remoteSegmentStoreDirectory)); + when(remoteStore.directory()).thenReturn(remoteStoreFilterDirectory); + } } From d537a4ecb79cf3b30c3c78e83faba6f0f5e8dd0b Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 25 Jul 2023 18:55:43 -0700 Subject: [PATCH 12/26] Spotless check fix Signed-off-by: Suraj Singh --- .../replication/RemoteStoreReplicationSource.java | 1 - ...icaRecoveryWithRemoteTranslogOnPrimaryTests.java | 8 -------- .../shard/SegmentReplicationIndexShardTests.java | 8 -------- ...entReplicationWithNodeToNodeIndexShardTests.java | 2 -- ...SegmentReplicationWithRemoteIndexShardTests.java | 9 --------- .../RemoteStorePeerRecoverySourceHandlerTests.java | 7 ------- .../RemoteStoreReplicationSourceTests.java | 13 +++---------- .../opensearch/index/shard/IndexShardTestCase.java | 11 ++++++----- 8 files changed, 9 insertions(+), 50 deletions(-) diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 614d9e5367c23..cf2973e3cb23b 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -25,7 +25,6 @@ import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index 982af2caa5d1c..0a2403603b87b 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -14,8 +14,6 @@ import org.opensearch.cluster.routing.RecoverySource; import org.opensearch.cluster.routing.ShardRouting; import org.opensearch.cluster.routing.ShardRoutingState; -import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexSettings; @@ -27,19 +25,13 @@ import org.opensearch.index.seqno.SequenceNumbers; import org.opensearch.index.store.Store; import org.opensearch.index.translog.WriteOnlyTranslogManager; -import org.opensearch.indices.IndicesService; -import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryTarget; -import org.opensearch.indices.replication.SegmentReplicationSourceFactory; -import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.common.ReplicationType; -import org.opensearch.transport.TransportService; import java.io.IOException; import java.nio.file.Path; import java.util.List; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Consumer; import static org.opensearch.cluster.routing.TestShardRouting.newShardRouting; diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 54cbcb9df2e55..024e4b5c0af0f 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -12,11 +12,9 @@ import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; import org.junit.Assert; -import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; -import org.opensearch.action.delete.DeleteRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; @@ -31,7 +29,6 @@ import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexSettings; -import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.Engine; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngine; @@ -45,7 +42,6 @@ import org.opensearch.index.translog.Translog; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; -import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.replication.CheckpointInfoResponse; import org.opensearch.indices.replication.GetSegmentFilesResponse; import org.opensearch.indices.replication.SegmentReplicationSource; @@ -57,8 +53,6 @@ import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.CopyState; import org.opensearch.indices.replication.common.ReplicationFailedException; -import org.opensearch.indices.replication.common.ReplicationListener; -import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -66,14 +60,12 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Function; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasToString; diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java index 50189d0808cd2..69876c2d1223d 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java @@ -744,6 +744,4 @@ public void testSegmentReplication_Index_Update_Delete() throws Exception { } } - - } diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java index 0cfc32b7963a8..b15d8b66fca55 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithRemoteIndexShardTests.java @@ -10,26 +10,17 @@ import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; -import org.opensearch.indices.IndicesService; -import org.opensearch.indices.recovery.RecoverySettings; -import org.opensearch.indices.replication.SegmentReplicationSourceFactory; -import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.common.ReplicationType; -import org.opensearch.transport.TransportService; import java.io.IOException; import java.util.List; -import java.util.function.Consumer; import static org.hamcrest.Matchers.equalTo; -import static org.mockito.Mockito.mock; public class SegmentReplicationWithRemoteIndexShardTests extends SegmentReplicationIndexShardTests { diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index cb9ed219034dd..249d57a2d6844 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -10,8 +10,6 @@ import org.junit.Before; import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.cluster.service.ClusterService; -import org.opensearch.common.settings.ClusterSettings; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexSettings; @@ -20,14 +18,9 @@ import org.opensearch.index.replication.OpenSearchIndexLevelReplicationTestCase; import org.opensearch.index.seqno.ReplicationTracker; import org.opensearch.index.shard.IndexShard; -import org.opensearch.indices.IndicesService; -import org.opensearch.indices.replication.SegmentReplicationSourceFactory; -import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.common.ReplicationType; -import org.opensearch.transport.TransportService; import java.nio.file.Path; -import java.util.function.Consumer; public class RemoteStorePeerRecoverySourceHandlerTests extends OpenSearchIndexLevelReplicationTestCase { diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index 921e83c46a96e..182e2583e4a0b 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -8,10 +8,7 @@ package org.opensearch.indices.replication; -import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.FilterDirectory; -import org.apache.lucene.util.Version; -import org.mockito.Mockito; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; @@ -58,11 +55,7 @@ public void setUp() throws Exception { indexDoc(primaryShard, "_doc", "1"); indexDoc(primaryShard, "_doc", "2"); primaryShard.refresh("test"); - replicaShard = newStartedShard( - false, - settings, - new NRTReplicationEngineFactory() - ); + replicaShard = newStartedShard(false, settings, new NRTReplicationEngineFactory()); } @Override @@ -78,7 +71,7 @@ public void testGetCheckpointMetadata() throws ExecutionException, InterruptedEx replicationSource.getCheckpointMetadata(REPLICATION_ID, checkpoint, res); CheckpointInfoResponse response = res.get(); assert (response.getCheckpoint().equals(checkpoint)); - assert (!response.getMetadataMap().isEmpty()); + assert (response.getMetadataMap().isEmpty() == false); } public void testGetCheckpointMetadataFailure() { @@ -116,7 +109,7 @@ public void testGetSegmentFilesFileAlreadyExists() throws IOException, Interrupt res.get(); } catch (Exception ex) { latch.countDown(); - assertTrue (ex.getCause() instanceof FileAlreadyExistsException); + assertTrue(ex.getCause() instanceof FileAlreadyExistsException); } latch.await(); } diff --git a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java index 7b497c7177ae5..66e5459cfea3b 100644 --- a/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/shard/IndexShardTestCase.java @@ -1326,11 +1326,12 @@ public static Engine.Warmer createTestWarmer(IndexSettings indexSettings) { }; } - private SegmentReplicationTargetService getSegmentReplicationTargetService(TransportService transportService, - IndicesService indicesService, - ClusterService clusterService, - SegmentReplicationSourceFactory sourceFactory - ) { + private SegmentReplicationTargetService getSegmentReplicationTargetService( + TransportService transportService, + IndicesService indicesService, + ClusterService clusterService, + SegmentReplicationSourceFactory sourceFactory + ) { return new SegmentReplicationTargetService( threadPool, new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), From dfdc7cc9d121fbe4ceff41ae3fce4a8cba160ea4 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Wed, 26 Jul 2023 18:10:10 -0700 Subject: [PATCH 13/26] Address review comments Signed-off-by: Suraj Singh --- .../java/org/opensearch/core/index/Index.java | 7 +++---- .../opensearch/core/index/shard/ShardId.java | 12 +++++------ .../metadata/RemoteSegmentMetadata.java | 4 ++-- .../RemoteStoreReplicationSource.java | 7 ++----- .../replication/SegmentReplicationTarget.java | 4 ++++ .../checkpoint/ReplicationCheckpoint.java | 20 ++++++++++--------- .../RemoteSegmentStoreDirectoryTests.java | 2 +- .../RemoteSegmentMetadataHandlerTests.java | 4 ++-- .../indices/recovery/AsyncRecoveryTarget.java | 5 +---- 9 files changed, 32 insertions(+), 33 deletions(-) diff --git a/libs/core/src/main/java/org/opensearch/core/index/Index.java b/libs/core/src/main/java/org/opensearch/core/index/Index.java index 50c3ccd2d5382..37440199100b8 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/Index.java +++ b/libs/core/src/main/java/org/opensearch/core/index/Index.java @@ -136,14 +136,13 @@ public static Index fromXContent(final XContentParser parser) throws IOException return INDEX_PARSER.parse(parser, null).build(); } - public void writeTo(IndexOutput out) throws IOException { + public void writeToIndexOutput(IndexOutput out) throws IOException { out.writeString(name); out.writeString(uuid); } - public Index(IndexInput in) throws IOException { - this.name = in.readString(); - this.uuid = in.readString(); + public static Index readFromIndexOutput(IndexInput in) throws IOException { + return new Index(in.readString(), in.readString()); } /** diff --git a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java index 9fb8c1d66b649..f0aa170af608f 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java +++ b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java @@ -150,14 +150,14 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder.value(toString()); } - public void writeTo(IndexOutput out) throws IOException { - index.writeTo(out); + public void writeToIndexOutput(IndexOutput out) throws IOException { + index.writeToIndexOutput(out); out.writeVInt(shardId); } - public ShardId(IndexInput in) throws IOException { - index = new Index(in); - shardId = in.readVInt(); - hashCode = computeHashCode(); + public static ShardId readFromIndexOutput(IndexInput in) throws IOException { + Index index = Index.readFromIndexOutput(in); + int shardId = in.readVInt(); + return new ShardId(index, shardId); } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 96e52e29a6687..863cc3c371b45 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -101,14 +101,14 @@ public static Map f public void write(IndexOutput out) throws IOException { out.writeMapOfStrings(toMapOfStrings()); - replicationCheckpoint.writeTo(out); + replicationCheckpoint.writeToIndexOutput(out); out.writeLong(segmentInfosBytes.length); out.writeBytes(segmentInfosBytes, segmentInfosBytes.length); } public static RemoteSegmentMetadata read(IndexInput indexInput) throws IOException { Map metadata = indexInput.readMapOfStrings(); - ReplicationCheckpoint replicationCheckpoint = new ReplicationCheckpoint(indexInput); + ReplicationCheckpoint replicationCheckpoint = ReplicationCheckpoint.readFromIndexOutput(indexInput); int byteArraySize = (int) indexInput.readLong(); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index cf2973e3cb23b..e7351b9bb4880 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -108,15 +108,12 @@ public void getSegmentFiles( indexShard.store().incRef(); indexShard.remoteStore().incRef(); final Directory storeDirectory = indexShard.store().directory(); - String segmentNFile = null; for (StoreFileMetadata fileMetadata : filesToFetch) { String file = fileMetadata.name(); + assert file.startsWith(IndexFileNames.SEGMENTS) == false + : "Segments_N file is not required for round of segment replication"; storeDirectory.copyFrom(remoteDirectory, file, file, IOContext.DEFAULT); downloadedSegments.add(fileMetadata); - if (file.startsWith(IndexFileNames.SEGMENTS)) { - assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; - segmentNFile = file; - } } storeDirectory.sync(downloadedSegments.stream().map(metadata -> metadata.name()).collect(Collectors.toList())); logger.trace("Downloaded segments from remote store {}", downloadedSegments); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index 334e364040218..c5405a368d7db 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -194,6 +194,10 @@ private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, throws OpenSearchCorruptionException { cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION); + // Handle empty SegmentInfos bytes for recovering replicas + if (checkpointInfoResponse.getInfosBytes() == null) { + return; + } Store store = null; try { store = store(); diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index e002c35a430ec..59ea100e98a82 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -189,8 +189,8 @@ public String toString() { + '}'; } - public void writeTo(IndexOutput out) throws IOException { - shardId.writeTo(out); + public void writeToIndexOutput(IndexOutput out) throws IOException { + shardId.writeToIndexOutput(out); out.writeLong(primaryTerm); out.writeLong(segmentsGen); out.writeLong(segmentInfosVersion); @@ -198,12 +198,14 @@ public void writeTo(IndexOutput out) throws IOException { out.writeString(codec); } - public ReplicationCheckpoint(IndexInput in) throws IOException { - shardId = new ShardId(in); - primaryTerm = in.readLong(); - segmentsGen = in.readLong(); - segmentInfosVersion = in.readLong(); - length = in.readLong(); - codec = in.readString(); + public static ReplicationCheckpoint readFromIndexOutput(IndexInput in) throws IOException { + return new ReplicationCheckpoint( + ShardId.readFromIndexOutput(in), + in.readLong(), + in.readLong(), + in.readLong(), + in.readLong(), + in.readString() + ); } } diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index 32a780484992f..bc0ce3615e751 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -273,7 +273,7 @@ private ByteArrayIndexInput createMetadataFileBytes(Map segmentF OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("segment metadata", "metadata output stream", output, 4096); CodecUtil.writeHeader(indexOutput, RemoteSegmentMetadata.METADATA_CODEC, RemoteSegmentMetadata.CURRENT_VERSION); indexOutput.writeMapOfStrings(segmentFilesMap); - replicationCheckpoint.writeTo(indexOutput); + replicationCheckpoint.writeToIndexOutput(indexOutput); indexOutput.writeLong(byteArray.length); indexOutput.writeBytes(byteArray, byteArray.length); CodecUtil.writeFooter(indexOutput); diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java index 581605961dfd8..c05eaf6f6c7d5 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java @@ -70,7 +70,7 @@ public void testReadContentNoSegmentInfos() throws IOException { OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); - replicationCheckpoint.writeTo(indexOutput); + replicationCheckpoint.writeToIndexOutput(indexOutput); indexOutput.writeLong(0); indexOutput.writeBytes(new byte[0], 0); indexOutput.close(); @@ -86,7 +86,7 @@ public void testReadContentWithSegmentInfos() throws IOException { OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); - replicationCheckpoint.writeTo(indexOutput); + replicationCheckpoint.writeToIndexOutput(indexOutput); ByteBuffersIndexOutput segmentInfosOutput = new ByteBuffersIndexOutput(new ByteBuffersDataOutput(), "test", "resource"); segmentInfos.write(segmentInfosOutput); byte[] segmentInfosBytes = segmentInfosOutput.toArrayCopy(); diff --git a/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java b/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java index 6f3b0ec622da1..b3398fb752be5 100644 --- a/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java +++ b/test/framework/src/main/java/org/opensearch/indices/recovery/AsyncRecoveryTarget.java @@ -90,10 +90,7 @@ public void prepareForTranslogOperations(int totalTranslogOps, ActionListener Date: Wed, 26 Jul 2023 20:07:39 -0700 Subject: [PATCH 14/26] Ignore files already in store while computing segment file diff with primary Signed-off-by: Suraj Singh --- .../org/opensearch/index/store/Store.java | 13 ++- .../RemoteStoreReplicationSource.java | 11 ++- .../replication/SegmentReplicationTarget.java | 10 +-- .../SegmentReplicationIndexShardTests.java | 81 +++++++++++++------ ...licationWithNodeToNodeIndexShardTests.java | 47 +---------- ...teStorePeerRecoverySourceHandlerTests.java | 4 - 6 files changed, 81 insertions(+), 85 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index 921deae41946a..5f663f8cce930 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -67,6 +67,7 @@ import org.opensearch.ExceptionsHelper; import org.opensearch.common.CheckedConsumer; import org.opensearch.common.UUIDs; +import org.opensearch.common.util.set.Sets; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; @@ -117,6 +118,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Consumer; +import java.util.stream.Collectors; import java.util.zip.CRC32; import java.util.zip.Checksum; @@ -388,11 +390,15 @@ public Map getSegmentMetadataMap(SegmentInfos segment * */ public static RecoveryDiff segmentReplicationDiff(Map source, Map target) { + return segmentReplicationDiff(source, target, false, null); + } + + public static RecoveryDiff segmentReplicationDiff(Map source, Map target, boolean includeSegmentNFile, Collection ignoreAdditionalFiles) { final List identical = new ArrayList<>(); final List different = new ArrayList<>(); - final List missing = new ArrayList<>(); + List missing = new ArrayList<>(); for (StoreFileMetadata value : source.values()) { - if (value.name().startsWith(IndexFileNames.SEGMENTS)) { + if (includeSegmentNFile == false && value.name().startsWith(IndexFileNames.SEGMENTS)) { continue; } if (target.containsKey(value.name()) == false) { @@ -407,6 +413,9 @@ public static RecoveryDiff segmentReplicationDiff(Map } } } + if (ignoreAdditionalFiles != null) { + missing = missing.stream().filter(metadata -> ignoreAdditionalFiles.contains(metadata.name()) == false).collect(Collectors.toList()); + } return new RecoveryDiff( Collections.unmodifiableList(identical), Collections.unmodifiableList(different), diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index e7351b9bb4880..80dc032463e5d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -25,6 +25,7 @@ import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; @@ -101,8 +102,11 @@ public void getSegmentFiles( return; } logger.trace("Downloading segments files from remote store {}", filesToFetch); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); List downloadedSegments = new ArrayList<>(); + String segmentNFile = null; + Collection directoryFiles = List.of(indexShard.store().directory().listAll()); if (remoteSegmentMetadata != null) { try { indexShard.store().incRef(); @@ -110,10 +114,13 @@ public void getSegmentFiles( final Directory storeDirectory = indexShard.store().directory(); for (StoreFileMetadata fileMetadata : filesToFetch) { String file = fileMetadata.name(); - assert file.startsWith(IndexFileNames.SEGMENTS) == false - : "Segments_N file is not required for round of segment replication"; + assert directoryFiles.contains(file) == false : "Local store already contains the file"; storeDirectory.copyFrom(remoteDirectory, file, file, IOContext.DEFAULT); downloadedSegments.add(fileMetadata); + if (file.startsWith(IndexFileNames.SEGMENTS)) { + assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; + segmentNFile = file; + } } storeDirectory.sync(downloadedSegments.stream().map(metadata -> metadata.name()).collect(Collectors.toList())); logger.trace("Downloaded segments from remote store {}", downloadedSegments); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index c5405a368d7db..3d13d6df40b51 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -52,10 +52,6 @@ public class SegmentReplicationTarget extends ReplicationTarget { public final static String REPLICATION_PREFIX = "replication."; - public ReplicationCheckpoint getCheckpoint() { - return this.checkpoint; - } - public SegmentReplicationTarget(IndexShard indexShard, SegmentReplicationSource source, ReplicationListener listener) { super("replication_target", indexShard, new ReplicationLuceneIndex(), listener); this.checkpoint = indexShard.getLatestReplicationCheckpoint(); @@ -114,6 +110,10 @@ public boolean reset(CancellableThreads newTargetCancellableThreads) throws IOEx return false; } + public ReplicationCheckpoint getCheckpoint() { + return this.checkpoint; + } + @Override public void writeFileChunk( StoreFileMetadata metadata, @@ -167,7 +167,7 @@ public void startReplication(ActionListener listener) { private List getFiles(CheckpointInfoResponse checkpointInfo) throws IOException { cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FILE_DIFF); - final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap()); + final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap(), indexShard.indexSettings().isRemoteStoreEnabled(), List.of(indexShard.store().directory().listAll())); logger.trace(() -> new ParameterizedMessage("Replication diff for checkpoint {} {}", checkpointInfo.getCheckpoint(), diff)); /* * Segments are immutable. So if the replica has any segments with the same name that differ from the one in the incoming diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 024e4b5c0af0f..7c6d165cf3498 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -12,9 +12,11 @@ import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.AlreadyClosedException; import org.junit.Assert; +import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; +import org.opensearch.action.delete.DeleteRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; @@ -29,7 +31,9 @@ import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexSettings; +import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.Engine; +import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; @@ -42,6 +46,7 @@ import org.opensearch.index.translog.Translog; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; +import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.replication.CheckpointInfoResponse; import org.opensearch.indices.replication.GetSegmentFilesResponse; import org.opensearch.indices.replication.SegmentReplicationSource; @@ -53,6 +58,8 @@ import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; import org.opensearch.indices.replication.common.CopyState; import org.opensearch.indices.replication.common.ReplicationFailedException; +import org.opensearch.indices.replication.common.ReplicationListener; +import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; @@ -66,8 +73,11 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Function; +import static java.util.Arrays.asList; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.mockito.Mockito.any; @@ -85,28 +95,6 @@ public class SegmentReplicationIndexShardTests extends OpenSearchIndexLevelRepli .put(IndexMetadata.SETTING_REPLICATION_TYPE, ReplicationType.SEGMENT) .build(); - protected SegmentReplicationTargetService segmentReplicationTargetService() { - RecoverySettings recoverySettings = new RecoverySettings( - Settings.EMPTY, - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS) - ); - TransportService transportService = mock(TransportService.class); - IndicesService indicesService = mock(IndicesService.class); - SegmentReplicationSourceFactory sourceFactory = new SegmentReplicationSourceFactory( - transportService, - recoverySettings, - clusterService - ); - return new SegmentReplicationTargetService( - threadPool, - new RecoverySettings(Settings.EMPTY, new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)), - mock(TransportService.class), - sourceFactory, - indicesService, - clusterService - ); - } - protected ReplicationGroup getReplicationGroup(int numberOfReplicas) throws IOException { return createGroup(numberOfReplicas, getIndexSettings(), indexMapping, new NRTReplicationEngineFactory()); } @@ -119,10 +107,6 @@ protected Settings getIndexSettings() { return settings; } - protected IndexShard getStartedShard() throws IOException { - return newStartedShard(randomBoolean(), settings, new NRTReplicationEngineFactory()); - } - /** * Validates happy path of segment replication where primary index docs which are replicated to replica shards. Assertions * made on doc count on both primary and replica. @@ -220,6 +204,51 @@ public void testSegmentInfosAndReplicationCheckpointTuple() throws Exception { } } + public void testPrimaryRelocationWithSegRepFailure() throws Exception { + final IndexShard primarySource = newStartedShard(true, getIndexSettings()); + int totalOps = randomInt(10); + for (int i = 0; i < totalOps; i++) { + indexDoc(primarySource, "_doc", Integer.toString(i)); + } + IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1)); + final IndexShard primaryTarget = newShard( + primarySource.routingEntry().getTargetRelocatingShard(), + getIndexSettings(), + new NRTReplicationEngineFactory() + ); + updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetadata()); + + Function, List> replicatePrimaryFunction = (shardList) -> { + try { + throw new IOException("Expected failure"); + } catch (IOException e) { + throw new RuntimeException(e); + } + }; + Exception e = expectThrows( + Exception.class, + () -> recoverReplica( + primaryTarget, + primarySource, + (primary, sourceNode) -> new RecoveryTarget(primary, sourceNode, new ReplicationListener() { + @Override + public void onDone(ReplicationState state) { + throw new AssertionError("recovery must fail"); + } + + @Override + public void onFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { + assertEquals(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), "Expected failure"); + } + }), + true, + true, + replicatePrimaryFunction + ) + ); + closeShards(primarySource, primaryTarget); + } + private void assertReplicationCheckpoint(IndexShard shard, SegmentInfos segmentInfos, ReplicationCheckpoint checkpoint) throws IOException { assertNotNull(segmentInfos); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java index 69876c2d1223d..d749060934661 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java @@ -434,6 +434,7 @@ public void testTemporaryFilesNotCleanup() throws Exception { } } + // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store public void testReplicaReceivesLowerGeneration() throws Exception { // when a replica gets incoming segments that are lower than what it currently has on disk. @@ -647,52 +648,6 @@ public void testReplicaRestarts() throws Exception { } } - // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store - public void testPrimaryRelocationWithSegRepFailure() throws Exception { - final IndexShard primarySource = newStartedShard(true, getIndexSettings()); - int totalOps = randomInt(10); - for (int i = 0; i < totalOps; i++) { - indexDoc(primarySource, "_doc", Integer.toString(i)); - } - IndexShardTestCase.updateRoutingEntry(primarySource, primarySource.routingEntry().relocate(randomAlphaOfLength(10), -1)); - final IndexShard primaryTarget = newShard( - primarySource.routingEntry().getTargetRelocatingShard(), - getIndexSettings(), - new NRTReplicationEngineFactory() - ); - updateMappings(primaryTarget, primarySource.indexSettings().getIndexMetadata()); - - Function, List> replicatePrimaryFunction = (shardList) -> { - try { - throw new IOException("Expected failure"); - } catch (IOException e) { - throw new RuntimeException(e); - } - }; - Exception e = expectThrows( - Exception.class, - () -> recoverReplica( - primaryTarget, - primarySource, - (primary, sourceNode) -> new RecoveryTarget(primary, sourceNode, new ReplicationListener() { - @Override - public void onDone(ReplicationState state) { - throw new AssertionError("recovery must fail"); - } - - @Override - public void onFailure(ReplicationState state, ReplicationFailedException e, boolean sendShardFailure) { - assertEquals(ExceptionsHelper.unwrap(e, IOException.class).getMessage(), "Expected failure"); - } - }), - true, - true, - replicatePrimaryFunction - ) - ); - closeShards(primarySource, primaryTarget); - } - // Todo: Move this test to SegmentReplicationIndexShardTests so that it runs for both node-node & remote store public void testSegmentReplication_Index_Update_Delete() throws Exception { String mappings = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": { \"properties\": { \"foo\": { \"type\": \"keyword\"} }}}"; diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index 249d57a2d6844..8bc993a0413c0 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -44,7 +44,6 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { final String indexMapping = "{ \"" + MapperService.SINGLE_MAPPING_NAME + "\": {} }"; try (ReplicationGroup shards = createGroup(0, settings, indexMapping, new NRTReplicationEngineFactory(), remoteDir)) { - logger.info("--> Start primary shard, index docs and flush"); // Step1 - Start primary, index docs and flush shards.startPrimary(); final IndexShard primary = shards.getPrimary(); @@ -52,13 +51,11 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { shards.flush(); // Step 2 - Start replica for recovery to happen, check both has same number of docs - logger.info("--> Add replica shard"); final IndexShard replica1 = shards.addReplica(remoteDir); shards.startAll(); assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); // Step 3 - Index more docs, run segment replication, check both have same number of docs - logger.info("--> Add more docs and verify"); int moreDocs = shards.indexDocs(randomIntBetween(10, 20)); primary.refresh("test"); replicateSegments(primary, shards.getReplicas()); @@ -73,7 +70,6 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { assertFalse(primary.getRetentionLeases().contains(ReplicationTracker.getPeerRecoveryRetentionLeaseId(replica1.routingEntry()))); // Step 6 - Start new replica, recovery happens, and check that new replica has all docs - logger.info("--> Add another replica shard"); final IndexShard replica2 = shards.addReplica(remoteDir); shards.startAll(); shards.assertAllEqual(numDocs + moreDocs); From 7b1b6ffc722f4d1e2d5ea21a214da3f8a5608ae5 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Wed, 26 Jul 2023 20:16:59 -0700 Subject: [PATCH 15/26] Spotless fix Signed-off-by: Suraj Singh --- .../main/java/org/opensearch/index/store/Store.java | 12 +++++++++--- .../replication/SegmentReplicationTarget.java | 7 ++++++- .../shard/SegmentReplicationIndexShardTests.java | 6 ------ ...mentReplicationWithNodeToNodeIndexShardTests.java | 5 ----- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index 5f663f8cce930..d2bb87e7cf0be 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -67,7 +67,6 @@ import org.opensearch.ExceptionsHelper; import org.opensearch.common.CheckedConsumer; import org.opensearch.common.UUIDs; -import org.opensearch.common.util.set.Sets; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.core.common.io.stream.StreamInput; @@ -393,7 +392,12 @@ public static RecoveryDiff segmentReplicationDiff(Map return segmentReplicationDiff(source, target, false, null); } - public static RecoveryDiff segmentReplicationDiff(Map source, Map target, boolean includeSegmentNFile, Collection ignoreAdditionalFiles) { + public static RecoveryDiff segmentReplicationDiff( + Map source, + Map target, + boolean includeSegmentNFile, + Collection ignoreAdditionalFiles + ) { final List identical = new ArrayList<>(); final List different = new ArrayList<>(); List missing = new ArrayList<>(); @@ -414,7 +418,9 @@ public static RecoveryDiff segmentReplicationDiff(Map } } if (ignoreAdditionalFiles != null) { - missing = missing.stream().filter(metadata -> ignoreAdditionalFiles.contains(metadata.name()) == false).collect(Collectors.toList()); + missing = missing.stream() + .filter(metadata -> ignoreAdditionalFiles.contains(metadata.name()) == false) + .collect(Collectors.toList()); } return new RecoveryDiff( Collections.unmodifiableList(identical), diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index 3d13d6df40b51..868452819a01d 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -167,7 +167,12 @@ public void startReplication(ActionListener listener) { private List getFiles(CheckpointInfoResponse checkpointInfo) throws IOException { cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FILE_DIFF); - final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap(), indexShard.indexSettings().isRemoteStoreEnabled(), List.of(indexShard.store().directory().listAll())); + final Store.RecoveryDiff diff = Store.segmentReplicationDiff( + checkpointInfo.getMetadataMap(), + indexShard.getSegmentMetadataMap(), + indexShard.indexSettings().isRemoteStoreEnabled(), + List.of(indexShard.store().directory().listAll()) + ); logger.trace(() -> new ParameterizedMessage("Replication diff for checkpoint {} {}", checkpointInfo.getCheckpoint(), diff)); /* * Segments are immutable. So if the replica has any segments with the same name that differ from the one in the incoming diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 7c6d165cf3498..8f03eecae97e6 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -16,7 +16,6 @@ import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest; -import org.opensearch.action.delete.DeleteRequest; import org.opensearch.action.index.IndexRequest; import org.opensearch.action.support.PlainActionFuture; import org.opensearch.cluster.metadata.IndexMetadata; @@ -31,9 +30,7 @@ import org.opensearch.common.xcontent.XContentType; import org.opensearch.common.lease.Releasable; import org.opensearch.index.IndexSettings; -import org.opensearch.index.engine.DocIdSeqNoAndSource; import org.opensearch.index.engine.Engine; -import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.engine.InternalEngineFactory; import org.opensearch.index.engine.NRTReplicationEngine; import org.opensearch.index.engine.NRTReplicationEngineFactory; @@ -44,7 +41,6 @@ import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.index.translog.SnapshotMatchers; import org.opensearch.index.translog.Translog; -import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.replication.CheckpointInfoResponse; @@ -75,9 +71,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; -import static java.util.Arrays.asList; import static org.hamcrest.Matchers.containsString; -import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.mockito.Mockito.any; diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java index d749060934661..69846fbbe1dd4 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationWithNodeToNodeIndexShardTests.java @@ -10,7 +10,6 @@ import org.apache.lucene.index.SegmentInfos; import org.junit.Assert; -import org.opensearch.ExceptionsHelper; import org.opensearch.action.ActionListener; import org.opensearch.action.admin.indices.flush.FlushRequest; import org.opensearch.action.delete.DeleteRequest; @@ -31,7 +30,6 @@ import org.opensearch.index.store.StoreFileMetadata; import org.opensearch.indices.IndicesService; import org.opensearch.indices.recovery.RecoverySettings; -import org.opensearch.indices.recovery.RecoveryTarget; import org.opensearch.indices.replication.CheckpointInfoResponse; import org.opensearch.indices.replication.GetSegmentFilesResponse; import org.opensearch.indices.replication.SegmentReplicationSource; @@ -39,9 +37,6 @@ import org.opensearch.indices.replication.SegmentReplicationTarget; import org.opensearch.indices.replication.SegmentReplicationTargetService; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; -import org.opensearch.indices.replication.common.ReplicationFailedException; -import org.opensearch.indices.replication.common.ReplicationListener; -import org.opensearch.indices.replication.common.ReplicationState; import org.opensearch.threadpool.ThreadPool; import org.opensearch.transport.TransportService; From 6ceaf1d6335f128592b3df7d65c2af222ef7dede Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Wed, 26 Jul 2023 22:19:45 -0700 Subject: [PATCH 16/26] Fix failing UT Signed-off-by: Suraj Singh --- .../ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java | 2 -- .../index/shard/SegmentReplicationIndexShardTests.java | 2 -- .../replication/RemoteStoreReplicationSourceTests.java | 7 ++++--- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java index 0a2403603b87b..20b3dfc0f93a6 100644 --- a/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java +++ b/server/src/test/java/org/opensearch/index/shard/ReplicaRecoveryWithRemoteTranslogOnPrimaryTests.java @@ -139,7 +139,6 @@ public void testNoTranslogHistoryTransferred() throws Exception { // Step1 - Start primary, index docs, flush, index more docs, check translog in primary as expected shards.startPrimary(); final IndexShard primary = shards.getPrimary(); - logger.info("--> Index docs on primary and flush"); int numDocs = shards.indexDocs(randomIntBetween(10, 100)); shards.flush(); List docIdAndSeqNosAfterFlush = getDocIdAndSeqNos(primary); @@ -147,7 +146,6 @@ public void testNoTranslogHistoryTransferred() throws Exception { assertEquals(numDocs + moreDocs, getTranslog(primary).totalOperations()); // Step 2 - Start replica, recovery happens, check docs recovered till last flush - logger.info("--> Add replica shard and start"); final IndexShard replica = shards.addReplica(remoteDir); shards.startAll(); assertEquals(docIdAndSeqNosAfterFlush, getDocIdAndSeqNos(replica)); diff --git a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java index 8f03eecae97e6..12b7341349442 100644 --- a/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java +++ b/server/src/test/java/org/opensearch/index/shard/SegmentReplicationIndexShardTests.java @@ -114,9 +114,7 @@ public void testReplication() throws Exception { // index and replicate segments to replica. int numDocs = randomIntBetween(10, 20); - logger.info("--> Index {} docs", numDocs); shards.indexDocs(numDocs); - logger.info("--> Refresh"); primaryShard.refresh("test"); flushShard(primaryShard); replicateSegments(primaryShard, List.of(replicaShard)); diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index 182e2583e4a0b..f168ef9b80911 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -98,7 +98,7 @@ public void testGetSegmentFiles() throws ExecutionException, InterruptedExceptio closeShards(replicaShard); } - public void testGetSegmentFilesFileAlreadyExists() throws IOException, InterruptedException { + public void testGetSegmentFilesAlreadyExists() throws IOException, InterruptedException { final ReplicationCheckpoint checkpoint = primaryShard.getLatestReplicationCheckpoint(); List filesToFetch = primaryShard.getSegmentMetadataMap().values().stream().collect(Collectors.toList()); CountDownLatch latch = new CountDownLatch(1); @@ -107,9 +107,10 @@ public void testGetSegmentFilesFileAlreadyExists() throws IOException, Interrupt replicationSource = new RemoteStoreReplicationSource(primaryShard); replicationSource.getSegmentFiles(REPLICATION_ID, checkpoint, filesToFetch, primaryShard, res); res.get(); - } catch (Exception ex) { + } catch (AssertionError | ExecutionException ex) { latch.countDown(); - assertTrue(ex.getCause() instanceof FileAlreadyExistsException); + assertTrue(ex instanceof AssertionError); + assertEquals("Local store already contains the file", ex.getMessage()); } latch.await(); } From 90e4bc826881e7d9d307b37cd9be4c6ee1d768a2 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Wed, 26 Jul 2023 22:45:36 -0700 Subject: [PATCH 17/26] Spotless fix Signed-off-by: Suraj Singh --- .../indices/replication/RemoteStoreReplicationSourceTests.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index f168ef9b80911..caba7e4a41016 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -25,7 +25,6 @@ import org.opensearch.indices.replication.common.ReplicationType; import java.io.IOException; -import java.nio.file.FileAlreadyExistsException; import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; From ac59af9724e7caf9987a92d0b96c7da93ca00786 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Thu, 27 Jul 2023 14:08:12 -0700 Subject: [PATCH 18/26] Move read/writes from IndexInput/Output to RemoteSegmentMetadata Signed-off-by: Suraj Singh --- .../java/org/opensearch/core/index/Index.java | 11 ----- .../opensearch/core/index/shard/ShardId.java | 13 ------ .../metadata/RemoteSegmentMetadata.java | 46 ++++++++++++++++++- .../checkpoint/ReplicationCheckpoint.java | 22 --------- .../RemoteSegmentStoreDirectoryTests.java | 2 +- .../RemoteSegmentMetadataHandlerTests.java | 4 +- 6 files changed, 47 insertions(+), 51 deletions(-) diff --git a/libs/core/src/main/java/org/opensearch/core/index/Index.java b/libs/core/src/main/java/org/opensearch/core/index/Index.java index 37440199100b8..fc5c5152a5500 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/Index.java +++ b/libs/core/src/main/java/org/opensearch/core/index/Index.java @@ -32,8 +32,6 @@ package org.opensearch.core.index; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; import org.opensearch.core.ParseField; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -136,15 +134,6 @@ public static Index fromXContent(final XContentParser parser) throws IOException return INDEX_PARSER.parse(parser, null).build(); } - public void writeToIndexOutput(IndexOutput out) throws IOException { - out.writeString(name); - out.writeString(uuid); - } - - public static Index readFromIndexOutput(IndexInput in) throws IOException { - return new Index(in.readString(), in.readString()); - } - /** * Builder for Index objects. Used by ObjectParser instances only. * diff --git a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java index f0aa170af608f..b01121c3f30d4 100644 --- a/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java +++ b/libs/core/src/main/java/org/opensearch/core/index/shard/ShardId.java @@ -32,8 +32,6 @@ package org.opensearch.core.index.shard; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; import org.opensearch.core.common.Strings; import org.opensearch.core.common.io.stream.StreamInput; import org.opensearch.core.common.io.stream.StreamOutput; @@ -149,15 +147,4 @@ public int compareTo(ShardId o) { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { return builder.value(toString()); } - - public void writeToIndexOutput(IndexOutput out) throws IOException { - index.writeToIndexOutput(out); - out.writeVInt(shardId); - } - - public static ShardId readFromIndexOutput(IndexInput in) throws IOException { - Index index = Index.readFromIndexOutput(in); - int shardId = in.readVInt(); - return new ShardId(index, shardId); - } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index 863cc3c371b45..d9b28787d850d 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -14,6 +14,8 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; +import org.opensearch.core.index.Index; +import org.opensearch.core.index.shard.ShardId; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint; @@ -101,17 +103,57 @@ public static Map f public void write(IndexOutput out) throws IOException { out.writeMapOfStrings(toMapOfStrings()); - replicationCheckpoint.writeToIndexOutput(out); + writeCheckpointToIndexOutput(replicationCheckpoint, out); out.writeLong(segmentInfosBytes.length); out.writeBytes(segmentInfosBytes, segmentInfosBytes.length); } public static RemoteSegmentMetadata read(IndexInput indexInput) throws IOException { Map metadata = indexInput.readMapOfStrings(); - ReplicationCheckpoint replicationCheckpoint = ReplicationCheckpoint.readFromIndexOutput(indexInput); + ReplicationCheckpoint replicationCheckpoint = readCheckpointFromIndexInput(indexInput); int byteArraySize = (int) indexInput.readLong(); byte[] segmentInfosBytes = new byte[byteArraySize]; indexInput.readBytes(segmentInfosBytes, 0, byteArraySize); return new RemoteSegmentMetadata(RemoteSegmentMetadata.fromMapOfStrings(metadata), segmentInfosBytes, replicationCheckpoint); } + + public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicationCheckpoint, IndexOutput out) throws IOException { + writeShardIdToIndexOutput(replicationCheckpoint.getShardId(), out); + out.writeLong(replicationCheckpoint.getPrimaryTerm()); + out.writeLong(replicationCheckpoint.getSegmentsGen()); + out.writeLong(replicationCheckpoint.getSegmentInfosVersion()); + out.writeLong(replicationCheckpoint.getLength()); + out.writeString(replicationCheckpoint.getCodec()); + } + + private static ReplicationCheckpoint readCheckpointFromIndexInput(IndexInput in) throws IOException { + return new ReplicationCheckpoint( + readShardIdFromIndexInput(in), + in.readLong(), + in.readLong(), + in.readLong(), + in.readLong(), + in.readString() + ); + } + + public static void writeShardIdToIndexOutput(ShardId shardId, IndexOutput out) throws IOException { + writeIndexToIndexOutput(shardId.getIndex(), out); + out.writeVInt(shardId.getId()); + } + + public static ShardId readShardIdFromIndexInput(IndexInput in) throws IOException { + Index index = readIndexFromIndexInput(in); + int shardId = in.readVInt(); + return new ShardId(index, shardId); + } + + public static void writeIndexToIndexOutput(Index index, IndexOutput out) throws IOException { + out.writeString(index.getName()); + out.writeString(index.getUUID()); + } + + public static Index readIndexFromIndexInput(IndexInput in) throws IOException { + return new Index(in.readString(), in.readString()); + } } diff --git a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java index 59ea100e98a82..7549f3450e7f2 100644 --- a/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java +++ b/server/src/main/java/org/opensearch/indices/replication/checkpoint/ReplicationCheckpoint.java @@ -8,8 +8,6 @@ package org.opensearch.indices.replication.checkpoint; -import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; import org.opensearch.Version; import org.opensearch.common.Nullable; import org.opensearch.core.common.io.stream.StreamInput; @@ -188,24 +186,4 @@ public String toString() { + codec + '}'; } - - public void writeToIndexOutput(IndexOutput out) throws IOException { - shardId.writeToIndexOutput(out); - out.writeLong(primaryTerm); - out.writeLong(segmentsGen); - out.writeLong(segmentInfosVersion); - out.writeLong(length); - out.writeString(codec); - } - - public static ReplicationCheckpoint readFromIndexOutput(IndexInput in) throws IOException { - return new ReplicationCheckpoint( - ShardId.readFromIndexOutput(in), - in.readLong(), - in.readLong(), - in.readLong(), - in.readLong(), - in.readString() - ); - } } diff --git a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java index bc0ce3615e751..3b2e33388925a 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -273,7 +273,7 @@ private ByteArrayIndexInput createMetadataFileBytes(Map segmentF OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("segment metadata", "metadata output stream", output, 4096); CodecUtil.writeHeader(indexOutput, RemoteSegmentMetadata.METADATA_CODEC, RemoteSegmentMetadata.CURRENT_VERSION); indexOutput.writeMapOfStrings(segmentFilesMap); - replicationCheckpoint.writeToIndexOutput(indexOutput); + RemoteSegmentMetadata.writeCheckpointToIndexOutput(replicationCheckpoint, indexOutput); indexOutput.writeLong(byteArray.length); indexOutput.writeBytes(byteArray, byteArray.length); CodecUtil.writeFooter(indexOutput); diff --git a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java index c05eaf6f6c7d5..d0136f04afd75 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadataHandlerTests.java @@ -70,7 +70,7 @@ public void testReadContentNoSegmentInfos() throws IOException { OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); - replicationCheckpoint.writeToIndexOutput(indexOutput); + RemoteSegmentMetadata.writeCheckpointToIndexOutput(replicationCheckpoint, indexOutput); indexOutput.writeLong(0); indexOutput.writeBytes(new byte[0], 0); indexOutput.close(); @@ -86,7 +86,7 @@ public void testReadContentWithSegmentInfos() throws IOException { OutputStreamIndexOutput indexOutput = new OutputStreamIndexOutput("dummy bytes", "dummy stream", output, 4096); Map expectedOutput = getDummyData(); indexOutput.writeMapOfStrings(expectedOutput); - replicationCheckpoint.writeToIndexOutput(indexOutput); + RemoteSegmentMetadata.writeCheckpointToIndexOutput(replicationCheckpoint, indexOutput); ByteBuffersIndexOutput segmentInfosOutput = new ByteBuffersIndexOutput(new ByteBuffersDataOutput(), "test", "resource"); segmentInfos.write(segmentInfosOutput); byte[] segmentInfosBytes = segmentInfosOutput.toArrayCopy(); From bc9bd8c917b7074699404804c931308ef2b0c23b Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Thu, 27 Jul 2023 14:35:12 -0700 Subject: [PATCH 19/26] Address review commnt Signed-off-by: Suraj Singh --- .../metadata/RemoteSegmentMetadata.java | 29 +++++-------------- 1 file changed, 7 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java index d9b28787d850d..15703a2c02b13 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java +++ b/server/src/main/java/org/opensearch/index/store/remote/metadata/RemoteSegmentMetadata.java @@ -118,7 +118,12 @@ public static RemoteSegmentMetadata read(IndexInput indexInput) throws IOExcepti } public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicationCheckpoint, IndexOutput out) throws IOException { - writeShardIdToIndexOutput(replicationCheckpoint.getShardId(), out); + ShardId shardId = replicationCheckpoint.getShardId(); + // Write ShardId + out.writeString(shardId.getIndex().getName()); + out.writeString(shardId.getIndex().getUUID()); + out.writeVInt(shardId.getId()); + // Write remaining checkpoint fields out.writeLong(replicationCheckpoint.getPrimaryTerm()); out.writeLong(replicationCheckpoint.getSegmentsGen()); out.writeLong(replicationCheckpoint.getSegmentInfosVersion()); @@ -128,7 +133,7 @@ public static void writeCheckpointToIndexOutput(ReplicationCheckpoint replicatio private static ReplicationCheckpoint readCheckpointFromIndexInput(IndexInput in) throws IOException { return new ReplicationCheckpoint( - readShardIdFromIndexInput(in), + new ShardId(new Index(in.readString(), in.readString()), in.readVInt()), in.readLong(), in.readLong(), in.readLong(), @@ -136,24 +141,4 @@ private static ReplicationCheckpoint readCheckpointFromIndexInput(IndexInput in) in.readString() ); } - - public static void writeShardIdToIndexOutput(ShardId shardId, IndexOutput out) throws IOException { - writeIndexToIndexOutput(shardId.getIndex(), out); - out.writeVInt(shardId.getId()); - } - - public static ShardId readShardIdFromIndexInput(IndexInput in) throws IOException { - Index index = readIndexFromIndexInput(in); - int shardId = in.readVInt(); - return new ShardId(index, shardId); - } - - public static void writeIndexToIndexOutput(Index index, IndexOutput out) throws IOException { - out.writeString(index.getName()); - out.writeString(index.getUUID()); - } - - public static Index readIndexFromIndexInput(IndexInput in) throws IOException { - return new Index(in.readString(), in.readString()); - } } From 1689fd23865b80a03353f0c9a7d26212b18adf5c Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Mon, 31 Jul 2023 17:14:44 -0700 Subject: [PATCH 20/26] Update recovery flow to perform commits during recovery Signed-off-by: Suraj Singh --- .../opensearch/index/shard/IndexShard.java | 41 +++++-------------- .../org/opensearch/index/store/Store.java | 17 +------- .../RemoteStoreReplicationSource.java | 7 ++-- .../replication/SegmentReplicationTarget.java | 16 ++------ ...teStorePeerRecoverySourceHandlerTests.java | 4 ++ .../RemoteStoreReplicationSourceTests.java | 2 +- ...enSearchIndexLevelReplicationTestCase.java | 3 +- 7 files changed, 25 insertions(+), 65 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index f1d12ef383fc7..9ce5150faf088 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -198,10 +198,9 @@ import java.nio.file.NoSuchFileException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.EnumSet; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Locale; @@ -1988,7 +1987,7 @@ private long recoverLocallyUpToGlobalCheckpoint() { final Optional safeCommit; final long globalCheckpoint; try { - final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); + final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(TRANSLOG_UUID_KEY); globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); safeCommit = store.findSafeIndexCommit(globalCheckpoint); } catch (org.apache.lucene.index.IndexNotFoundException e) { @@ -2088,7 +2087,7 @@ private long recoverLocallyUptoLastCommit() { try { seqNo = Long.parseLong(store.readLastCommittedSegmentsInfo().getUserData().get(MAX_SEQ_NO)); } catch (org.apache.lucene.index.IndexNotFoundException e) { - logger.error("skip local recovery as no index commit found", e); + logger.error("skip local recovery as no index commit found"); return UNASSIGNED_SEQ_NO; } catch (Exception e) { logger.error("skip local recovery as failed to find the safe commit", e); @@ -2242,7 +2241,7 @@ private void loadGlobalCheckpointToReplicationTracker() throws IOException { // we have to set it before we open an engine and recover from the translog because // acquiring a snapshot from the translog causes a sync which causes the global checkpoint to be pulled in, // and an engine can be forced to close in ctor which also causes the global checkpoint to be pulled in. - final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); + final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(TRANSLOG_UUID_KEY); final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); replicationTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "read from translog checkpoint"); } @@ -4645,7 +4644,6 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re } else { storeDirectory = store.directory(); } - Set localSegmentFiles = Sets.newHashSet(storeDirectory.listAll()); copySegmentFiles(storeDirectory, remoteDirectory, null, uploadedSegments, overrideLocal); if (refreshLevelSegmentSync && remoteSegmentMetadata != null) { @@ -4659,32 +4657,15 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re indexInput, remoteSegmentMetadata.getGeneration() ); - // Replicas never need a local commit - if (this.shardRouting.primary()) { - long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); - // Following code block makes sure to use SegmentInfosSnapshot in the remote store if generation differs - // with local filesystem. If local filesystem already has segments_N+2 and infosSnapshot has generation N, - // after commit, there would be 2 files that would be created segments_N+1 and segments_N+2. With the - // policy of preserving only the latest commit, we will delete segments_N+1 which in fact is the part of the - // latest commit. - Optional localMaxSegmentInfos = localSegmentFiles.stream() - .filter(file -> file.startsWith(IndexFileNames.SEGMENTS)) - .max(Comparator.comparingLong(SegmentInfos::generationFromSegmentsFileName)); - if (localMaxSegmentInfos.isPresent() - && infosSnapshot.getGeneration() < SegmentInfos.generationFromSegmentsFileName(localMaxSegmentInfos.get()) - - 1) { - // If remote translog is not enabled, local translog will be created with different UUID. - // This fails in Store.trimUnsafeCommits() as translog UUID of checkpoint and SegmentInfos needs - // to be same. Following code block make sure to have the same UUID. - if (indexSettings.isRemoteTranslogStoreEnabled() == false) { - SegmentInfos localSegmentInfos = store.readLastCommittedSegmentsInfo(); - Map userData = new HashMap<>(infosSnapshot.getUserData()); - userData.put(TRANSLOG_UUID_KEY, localSegmentInfos.userData.get(TRANSLOG_UUID_KEY)); - infosSnapshot.setUserData(userData, false); + long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); + store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); + Collection currentInfoFiles = infosSnapshot.files(true); + if (recoveryState.getStage() != RecoveryState.Stage.DONE) { + for (String localFile : storeDirectory.listAll()) { + if (Store.isAutogenerated(localFile) == false && currentInfoFiles.contains(localFile) == false) { + storeDirectory.deleteFile(localFile); } - storeDirectory.deleteFile(localMaxSegmentInfos.get()); } - store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); } } } diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index d2bb87e7cf0be..d1ca74948012b 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -117,7 +117,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.function.Consumer; -import java.util.stream.Collectors; import java.util.zip.CRC32; import java.util.zip.Checksum; @@ -389,20 +388,11 @@ public Map getSegmentMetadataMap(SegmentInfos segment * */ public static RecoveryDiff segmentReplicationDiff(Map source, Map target) { - return segmentReplicationDiff(source, target, false, null); - } - - public static RecoveryDiff segmentReplicationDiff( - Map source, - Map target, - boolean includeSegmentNFile, - Collection ignoreAdditionalFiles - ) { final List identical = new ArrayList<>(); final List different = new ArrayList<>(); List missing = new ArrayList<>(); for (StoreFileMetadata value : source.values()) { - if (includeSegmentNFile == false && value.name().startsWith(IndexFileNames.SEGMENTS)) { + if (value.name().startsWith(IndexFileNames.SEGMENTS)) { continue; } if (target.containsKey(value.name()) == false) { @@ -417,11 +407,6 @@ public static RecoveryDiff segmentReplicationDiff( } } } - if (ignoreAdditionalFiles != null) { - missing = missing.stream() - .filter(metadata -> ignoreAdditionalFiles.contains(metadata.name()) == false) - .collect(Collectors.toList()); - } return new RecoveryDiff( Collections.unmodifiableList(identical), Collections.unmodifiableList(different), diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 80dc032463e5d..1c366076afe81 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -60,7 +60,7 @@ public void getCheckpointMetadata( // TODO: Need to figure out a way to pass this information for segment metadata via remote store. final Version version = indexShard.getSegmentInfosSnapshot().get().getCommitLuceneVersion(); try { - RemoteSegmentMetadata mdFile = remoteDirectory.readLatestMetadataFile(); + RemoteSegmentMetadata mdFile = remoteDirectory.init(); // During initial recovery flow, the remote store might not have metadata as primary hasn't uploaded anything yet. if (mdFile == null && indexShard.state().equals(IndexShardState.STARTED) == false) { listener.onResponse(new CheckpointInfoResponse(checkpoint, Collections.emptyMap(), null)); @@ -103,7 +103,7 @@ public void getSegmentFiles( } logger.trace("Downloading segments files from remote store {}", filesToFetch); - RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.init(); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); List downloadedSegments = new ArrayList<>(); String segmentNFile = null; Collection directoryFiles = List.of(indexShard.store().directory().listAll()); @@ -114,7 +114,7 @@ public void getSegmentFiles( final Directory storeDirectory = indexShard.store().directory(); for (StoreFileMetadata fileMetadata : filesToFetch) { String file = fileMetadata.name(); - assert directoryFiles.contains(file) == false : "Local store already contains the file"; + assert directoryFiles.contains(file) == false : "Local store already contains the file " + file; storeDirectory.copyFrom(remoteDirectory, file, file, IOContext.DEFAULT); downloadedSegments.add(fileMetadata); if (file.startsWith(IndexFileNames.SEGMENTS)) { @@ -122,7 +122,6 @@ public void getSegmentFiles( segmentNFile = file; } } - storeDirectory.sync(downloadedSegments.stream().map(metadata -> metadata.name()).collect(Collectors.toList())); logger.trace("Downloaded segments from remote store {}", downloadedSegments); } finally { indexShard.store().decRef(); diff --git a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java index 868452819a01d..c22701dfc94ce 100644 --- a/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java +++ b/server/src/main/java/org/opensearch/indices/replication/SegmentReplicationTarget.java @@ -32,7 +32,6 @@ import org.opensearch.indices.replication.common.ReplicationTarget; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Map; @@ -167,12 +166,7 @@ public void startReplication(ActionListener listener) { private List getFiles(CheckpointInfoResponse checkpointInfo) throws IOException { cancellableThreads.checkForCancel(); state.setStage(SegmentReplicationState.Stage.FILE_DIFF); - final Store.RecoveryDiff diff = Store.segmentReplicationDiff( - checkpointInfo.getMetadataMap(), - indexShard.getSegmentMetadataMap(), - indexShard.indexSettings().isRemoteStoreEnabled(), - List.of(indexShard.store().directory().listAll()) - ); + final Store.RecoveryDiff diff = Store.segmentReplicationDiff(checkpointInfo.getMetadataMap(), indexShard.getSegmentMetadataMap()); logger.trace(() -> new ParameterizedMessage("Replication diff for checkpoint {} {}", checkpointInfo.getCheckpoint(), diff)); /* * Segments are immutable. So if the replica has any segments with the same name that differ from the one in the incoming @@ -209,11 +203,9 @@ private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse, store.incRef(); Map tempFileNames; if (this.indexShard.indexSettings().isRemoteStoreEnabled() == true) { - tempFileNames = getSegmentFilesResponse.getFiles() != null - ? getSegmentFilesResponse.getFiles() - .stream() - .collect(Collectors.toMap(StoreFileMetadata::name, StoreFileMetadata::name)) - : Collections.emptyMap(); + tempFileNames = getSegmentFilesResponse.getFiles() + .stream() + .collect(Collectors.toMap(StoreFileMetadata::name, StoreFileMetadata::name)); } else { tempFileNames = multiFileWriter.getTempFileNames(); } diff --git a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java index 8bc993a0413c0..40182a85608ea 100644 --- a/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java +++ b/server/src/test/java/org/opensearch/indices/recovery/RemoteStorePeerRecoverySourceHandlerTests.java @@ -48,16 +48,19 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { shards.startPrimary(); final IndexShard primary = shards.getPrimary(); int numDocs = shards.indexDocs(randomIntBetween(10, 20)); + logger.info("--> Index numDocs {} and flush", numDocs); shards.flush(); // Step 2 - Start replica for recovery to happen, check both has same number of docs final IndexShard replica1 = shards.addReplica(remoteDir); + logger.info("--> Added and started replica {}", replica1.routingEntry()); shards.startAll(); assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); // Step 3 - Index more docs, run segment replication, check both have same number of docs int moreDocs = shards.indexDocs(randomIntBetween(10, 20)); primary.refresh("test"); + logger.info("--> Index more docs {} and replicate segments", moreDocs); replicateSegments(primary, shards.getReplicas()); assertEquals(getDocIdAndSeqNos(primary), getDocIdAndSeqNos(replica1)); @@ -71,6 +74,7 @@ public void testReplicaShardRecoveryUptoLastFlushedCommit() throws Exception { // Step 6 - Start new replica, recovery happens, and check that new replica has all docs final IndexShard replica2 = shards.addReplica(remoteDir); + logger.info("--> Added and started replica {}", replica2.routingEntry()); shards.startAll(); shards.assertAllEqual(numDocs + moreDocs); diff --git a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java index caba7e4a41016..9204f48ba5bdd 100644 --- a/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java +++ b/server/src/test/java/org/opensearch/indices/replication/RemoteStoreReplicationSourceTests.java @@ -109,7 +109,7 @@ public void testGetSegmentFilesAlreadyExists() throws IOException, InterruptedEx } catch (AssertionError | ExecutionException ex) { latch.countDown(); assertTrue(ex instanceof AssertionError); - assertEquals("Local store already contains the file", ex.getMessage()); + assertTrue(ex.getMessage().startsWith("Local store already contains the file")); } latch.await(); } diff --git a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java index b7e1460e22443..278847e56e65f 100644 --- a/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java +++ b/test/framework/src/main/java/org/opensearch/index/replication/OpenSearchIndexLevelReplicationTestCase.java @@ -150,8 +150,7 @@ protected ReplicationGroup createGroup(int replicas, Settings settings, EngineFa protected ReplicationGroup createGroup(int replicas, Settings settings, String mappings, EngineFactory engineFactory) throws IOException { Path remotePath = null; - if (settings.get(IndexMetadata.SETTING_REMOTE_STORE_ENABLED) != null - && settings.get(IndexMetadata.SETTING_REMOTE_STORE_ENABLED).equals("true")) { + if ("true".equals(settings.get(IndexMetadata.SETTING_REMOTE_STORE_ENABLED))) { remotePath = createTempDir(); } return createGroup(replicas, settings, mappings, engineFactory, remotePath); From dc795934f76aa80089654df6e20f16203b78d30e Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Mon, 31 Jul 2023 17:23:44 -0700 Subject: [PATCH 21/26] Remove un-necessary char Signed-off-by: Suraj Singh --- .../org/opensearch/index/store/RemoteSegmentStoreDirectory.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java index 2b5d3ff822757..8dfdb3e2c8e06 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -201,7 +201,7 @@ public RemoteSegmentMetadata readLatestMetadataFile() throws IOException { return remoteSegmentMetadata; } - private RemoteSegmentMetadata readMetadataFile(String metadataFilename) throws IOException {S + private RemoteSegmentMetadata readMetadataFile(String metadataFilename) throws IOException { try (IndexInput indexInput = remoteMetadataDirectory.openInput(metadataFilename, IOContext.DEFAULT)) { byte[] metadataBytes = new byte[(int) indexInput.length()]; indexInput.readBytes(metadataBytes, 0, (int) indexInput.length()); From 330e71216473f2400e78b34984ecf4dba794c42c Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Mon, 31 Jul 2023 18:32:48 -0700 Subject: [PATCH 22/26] Address review comments Signed-off-by: Suraj Singh --- .../src/main/java/org/opensearch/index/shard/IndexShard.java | 3 ++- server/src/main/java/org/opensearch/index/store/Store.java | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 9ce5150faf088..051543a1a30c2 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4660,7 +4660,8 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); Collection currentInfoFiles = infosSnapshot.files(true); - if (recoveryState.getStage() != RecoveryState.Stage.DONE) { + // For recovering shards, clean up un-usable incremental refresh files. + if (state() != IndexShardState.STARTED) { for (String localFile : storeDirectory.listAll()) { if (Store.isAutogenerated(localFile) == false && currentInfoFiles.contains(localFile) == false) { storeDirectory.deleteFile(localFile); diff --git a/server/src/main/java/org/opensearch/index/store/Store.java b/server/src/main/java/org/opensearch/index/store/Store.java index d1ca74948012b..921deae41946a 100644 --- a/server/src/main/java/org/opensearch/index/store/Store.java +++ b/server/src/main/java/org/opensearch/index/store/Store.java @@ -390,7 +390,7 @@ public Map getSegmentMetadataMap(SegmentInfos segment public static RecoveryDiff segmentReplicationDiff(Map source, Map target) { final List identical = new ArrayList<>(); final List different = new ArrayList<>(); - List missing = new ArrayList<>(); + final List missing = new ArrayList<>(); for (StoreFileMetadata value : source.values()) { if (value.name().startsWith(IndexFileNames.SEGMENTS)) { continue; From 0ac181a5b81443869c8661b2670da8ea88e15a42 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Mon, 31 Jul 2023 21:49:13 -0700 Subject: [PATCH 23/26] Update comment nit-pick Signed-off-by: Suraj Singh --- server/src/main/java/org/opensearch/index/shard/IndexShard.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 051543a1a30c2..8b600cf14c40d 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4660,7 +4660,7 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); Collection currentInfoFiles = infosSnapshot.files(true); - // For recovering shards, clean up un-usable incremental refresh files. + // For recovering shards, clean up un-usable incremental refresh files and commit points. if (state() != IndexShardState.STARTED) { for (String localFile : storeDirectory.listAll()) { if (Store.isAutogenerated(localFile) == false && currentInfoFiles.contains(localFile) == false) { From 4337dcd1ee09e2b2cf4d21d3bc7c291f9ec429a4 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 1 Aug 2023 12:37:42 -0700 Subject: [PATCH 24/26] Remove deletion logic causing read issues due to deleted segments_N Signed-off-by: Suraj Singh --- .../main/java/org/opensearch/index/shard/IndexShard.java | 9 --------- 1 file changed, 9 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 8b600cf14c40d..12bc507d9eda7 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -4659,15 +4659,6 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, boolean re ); long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY)); store.commitSegmentInfos(infosSnapshot, processedLocalCheckpoint, processedLocalCheckpoint); - Collection currentInfoFiles = infosSnapshot.files(true); - // For recovering shards, clean up un-usable incremental refresh files and commit points. - if (state() != IndexShardState.STARTED) { - for (String localFile : storeDirectory.listAll()) { - if (Store.isAutogenerated(localFile) == false && currentInfoFiles.contains(localFile) == false) { - storeDirectory.deleteFile(localFile); - } - } - } } } } catch (IOException e) { From 14ca8edab677316a32bf501067705da80cd4874f Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 1 Aug 2023 13:09:50 -0700 Subject: [PATCH 25/26] Spotless fix Signed-off-by: Suraj Singh --- server/src/main/java/org/opensearch/index/shard/IndexShard.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/shard/IndexShard.java b/server/src/main/java/org/opensearch/index/shard/IndexShard.java index 12bc507d9eda7..2b85193275a13 100644 --- a/server/src/main/java/org/opensearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/opensearch/index/shard/IndexShard.java @@ -198,7 +198,6 @@ import java.nio.file.NoSuchFileException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashSet; From 753729ca614100dc5d7a8d8f413b2b5ae36fc1a9 Mon Sep 17 00:00:00 2001 From: Suraj Singh Date: Tue, 1 Aug 2023 19:43:58 -0700 Subject: [PATCH 26/26] Fix unit tests Signed-off-by: Suraj Singh --- .../org/opensearch/remotestore/RemoteStoreStatsIT.java | 8 ++++---- .../snapshots/RemoteIndexSnapshotStatusApiIT.java | 1 + .../indices/replication/RemoteStoreReplicationSource.java | 6 ------ 3 files changed, 5 insertions(+), 10 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java index 840e3a07ed255..1c7f14701b3e7 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/RemoteStoreStatsIT.java @@ -272,12 +272,12 @@ public void testDownloadStatsCorrectnessSinglePrimarySingleReplica() throws Exce assertTrue( replicaStats.directoryFileTransferTrackerStats.transferredBytesStarted > 0 && primaryStats.uploadBytesStarted - - zeroStatePrimaryStats.uploadBytesStarted == replicaStats.directoryFileTransferTrackerStats.transferredBytesStarted + - zeroStatePrimaryStats.uploadBytesStarted >= replicaStats.directoryFileTransferTrackerStats.transferredBytesStarted ); assertTrue( replicaStats.directoryFileTransferTrackerStats.transferredBytesSucceeded > 0 && primaryStats.uploadBytesSucceeded - - zeroStatePrimaryStats.uploadBytesSucceeded == replicaStats.directoryFileTransferTrackerStats.transferredBytesSucceeded + - zeroStatePrimaryStats.uploadBytesSucceeded >= replicaStats.directoryFileTransferTrackerStats.transferredBytesSucceeded ); // Assert zero failures assertEquals(0, primaryStats.uploadBytesFailed); @@ -369,8 +369,8 @@ public void testDownloadStatsCorrectnessSinglePrimaryMultipleReplicaShards() thr assertEquals(0, uploadsFailed); assertEquals(0, uploadBytesFailed); for (int j = 0; j < response.getSuccessfulShards() - 1; j++) { - assertEquals(uploadBytesStarted - zeroStatePrimaryStats.uploadBytesStarted, (long) downloadBytesStarted.get(j)); - assertEquals(uploadBytesSucceeded - zeroStatePrimaryStats.uploadBytesSucceeded, (long) downloadBytesSucceeded.get(j)); + assertTrue(uploadBytesStarted - zeroStatePrimaryStats.uploadBytesStarted > downloadBytesStarted.get(j)); + assertTrue(uploadBytesSucceeded - zeroStatePrimaryStats.uploadBytesSucceeded > downloadBytesSucceeded.get(j)); assertEquals(0, (long) downloadBytesFailed.get(j)); } }, 60, TimeUnit.SECONDS); diff --git a/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java b/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java index b6a5188c99335..d17410d8921ed 100644 --- a/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/snapshots/RemoteIndexSnapshotStatusApiIT.java @@ -57,6 +57,7 @@ protected Settings nodeSettings(int nodeOrdinal) { .put(super.nodeSettings(nodeOrdinal)) .put(ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING.getKey(), 0) // We have tests that check by-timestamp order .put(FeatureFlags.REMOTE_STORE, "true") + .put(FeatureFlags.SEGMENT_REPLICATION_EXPERIMENTAL, "true") .put(remoteStoreClusterSettings("remote-store-repo-name")) .build(); } diff --git a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java index 1c366076afe81..7f444d0031533 100644 --- a/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java +++ b/server/src/main/java/org/opensearch/indices/replication/RemoteStoreReplicationSource.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.lucene.index.IndexFileNames; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; @@ -105,7 +104,6 @@ public void getSegmentFiles( RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); List downloadedSegments = new ArrayList<>(); - String segmentNFile = null; Collection directoryFiles = List.of(indexShard.store().directory().listAll()); if (remoteSegmentMetadata != null) { try { @@ -117,10 +115,6 @@ public void getSegmentFiles( assert directoryFiles.contains(file) == false : "Local store already contains the file " + file; storeDirectory.copyFrom(remoteDirectory, file, file, IOContext.DEFAULT); downloadedSegments.add(fileMetadata); - if (file.startsWith(IndexFileNames.SEGMENTS)) { - assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file"; - segmentNFile = file; - } } logger.trace("Downloaded segments from remote store {}", downloadedSegments); } finally {