Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Remote Store] Add support to separate segment infos snapshot from metadata #13114

Open
wants to merge 9 commits into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGELOG.md
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/),
- Add an individual setting of rate limiter for segment replication ([#12959](https://github.com/opensearch-project/OpenSearch/pull/12959))
- [Streaming Indexing] Ensure support of the new transport by security plugin ([#13174](https://github.com/opensearch-project/OpenSearch/pull/13174))
- Add cluster setting to dynamically configure the buckets for filter rewrite optimization. ([#13179](https://github.com/opensearch-project/OpenSearch/pull/13179))
- [Remote Store] Add support to separate segment infos snapshot from metadata ([#13114](https://github.com/opensearch-project/OpenSearch/pull/13114))

### Dependencies
- Bump `org.apache.commons:commons-configuration2` from 2.10.0 to 2.10.1 ([#12896](https://github.com/opensearch-project/OpenSearch/pull/12896))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.opensearch.index.mapper.MapperService;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.indices.IndicesService;
import org.opensearch.indices.RemoteStoreSettings;
import org.opensearch.indices.replication.common.ReplicationType;
import org.opensearch.repositories.RepositoriesService;
import org.opensearch.repositories.blobstore.BlobStoreRepository;
Expand Down Expand Up @@ -135,14 +136,17 @@ protected Settings nodeSettings(int nodeOrdinal) {
segmentRepoPath = randomRepoPath().toAbsolutePath();
translogRepoPath = randomRepoPath().toAbsolutePath();
}
Settings.Builder settingsBuilder = Settings.builder();
if (clusterSettingsSuppliedByTest) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal)).build();
settingsBuilder.put(super.nodeSettings(nodeOrdinal));
} else {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put(remoteStoreClusterSettings(REPOSITORY_NAME, segmentRepoPath, REPOSITORY_2_NAME, translogRepoPath))
.build();
settingsBuilder.put(super.nodeSettings(nodeOrdinal))
.put(remoteStoreClusterSettings(REPOSITORY_NAME, segmentRepoPath, REPOSITORY_2_NAME, translogRepoPath));
}
if (randomBoolean()) {
settingsBuilder.put(RemoteStoreSettings.CLUSTER_REMOTE_SEGMENT_SEPARATE_METADATA_SEGMENTINFOS_SETTING.getKey(), true);
Comment on lines +146 to +147
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@sachinpkale just confirming this segment infos snapshot is different than segments_N file which gets created at the commit time. this is the one which have latest segment info at the time of refresh?

}
return settingsBuilder.build();
}

protected void setFailRate(String repoName, int value) throws ExecutionException, InterruptedException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.opensearch.index.IndexSettings;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.IndexShardClosedException;
import org.opensearch.index.store.RemoteSegmentStoreDirectory;
import org.opensearch.index.translog.Translog.Durability;
import org.opensearch.indices.IndicesService;
import org.opensearch.indices.RemoteStoreSettings;
Expand Down Expand Up @@ -585,7 +586,9 @@ public void testFallbackToNodeToNodeSegmentCopy() throws Exception {
try (Stream<Path> files = Files.list(segmentDataPath)) {
files.forEach(p -> {
try {
Files.delete(p);
if (p.getFileName().toString().startsWith(RemoteSegmentStoreDirectory.SEGMENT_INFOS_SNAPSHOT_PREFIX) == false) {
Files.delete(p);
}
} catch (IOException e) {
// Ignore
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -732,7 +732,8 @@ public void apply(Settings value, Settings current, Settings previous) {

RemoteStoreSettings.CLUSTER_REMOTE_INDEX_SEGMENT_METADATA_RETENTION_MAX_COUNT_SETTING,
RemoteStoreSettings.CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING,
RemoteStoreSettings.CLUSTER_REMOTE_TRANSLOG_TRANSFER_TIMEOUT_SETTING
RemoteStoreSettings.CLUSTER_REMOTE_TRANSLOG_TRANSFER_TIMEOUT_SETTING,
RemoteStoreSettings.CLUSTER_REMOTE_SEGMENT_SEPARATE_METADATA_SEGMENTINFOS_SETTING
)
)
);
Expand Down
29 changes: 24 additions & 5 deletions server/src/main/java/org/opensearch/index/shard/IndexShard.java
Original file line number Diff line number Diff line change
Expand Up @@ -5048,15 +5048,34 @@ public void syncSegmentsFromRemoteSegmentStore(boolean overrideLocal, final Runn
copySegmentFiles(storeDirectory, remoteDirectory, null, uploadedSegments, overrideLocal, onFileSync);

if (remoteSegmentMetadata != null) {
final SegmentInfos infosSnapshot = store.buildSegmentInfos(
remoteSegmentMetadata.getSegmentInfosBytes(),
remoteSegmentMetadata.getGeneration()
);
final SegmentInfos infosSnapshot;
if (remoteSegmentMetadata.getSegmentInfosBytes().length == 0) {
List<String> segmentInfosSnapshotFilenames = Arrays.stream(store.directory().listAll())
.filter(file -> file.startsWith(RemoteSegmentStoreDirectory.SEGMENT_INFOS_SNAPSHOT_PREFIX))
.collect(Collectors.toList());
assert segmentInfosSnapshotFilenames.size() == 1;
try (
ChecksumIndexInput segmentInfosInput = store.directory()
.openChecksumInput(segmentInfosSnapshotFilenames.get(0), IOContext.READ)
) {
infosSnapshot = SegmentInfos.readCommit(
store.directory(),
segmentInfosInput,
remoteSegmentMetadata.getGeneration()
);
}
} else {
infosSnapshot = store.buildSegmentInfos(
remoteSegmentMetadata.getSegmentInfosBytes(),
remoteSegmentMetadata.getGeneration()
);
}
long processedLocalCheckpoint = Long.parseLong(infosSnapshot.getUserData().get(LOCAL_CHECKPOINT_KEY));
// delete any other commits, we want to start the engine only from a new commit made with the downloaded infos bytes.
// Extra segments will be wiped on engine open.
for (String file : List.of(store.directory().listAll())) {
if (file.startsWith(IndexFileNames.SEGMENTS)) {
if (file.startsWith(IndexFileNames.SEGMENTS)
|| file.startsWith(RemoteSegmentStoreDirectory.SEGMENT_INFOS_SNAPSHOT_PREFIX)) {
store.deleteQuiet(file);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -401,7 +401,8 @@ void uploadMetadata(Collection<String> localSegmentsPostRefresh, SegmentInfos se
storeDirectory,
translogFileGeneration,
replicationCheckpoint,
indexShard.getNodeId()
indexShard.getNodeId(),
indexShard.getRemoteStoreSettings().getClusterRemoteSegmentSeparateMetadataSegmentInfos()
);
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,8 @@ public final class RemoteSegmentStoreDirectory extends FilterDirectory implement
*/
public static final String SEGMENT_NAME_UUID_SEPARATOR = "__";

public static final String SEGMENT_INFOS_SNAPSHOT_PREFIX = "segment_infos_snapshot";

/**
* remoteDataDirectory is used to store segment files at path: cluster_UUID/index_UUID/shardId/segments/data
*/
Expand Down Expand Up @@ -136,6 +138,30 @@ public RemoteSegmentStoreDirectory(
init();
}

// Visible for testing
public static String getMetadataFilename(
String separator,
String prefix,
long primaryTerm,
long generation,
long translogGeneration,
long uploadCounter,
int metadataVersion,
String nodeId
) {
return String.join(
separator,
prefix,
RemoteStoreUtils.invertLong(primaryTerm),
RemoteStoreUtils.invertLong(generation),
RemoteStoreUtils.invertLong(translogGeneration),
RemoteStoreUtils.invertLong(uploadCounter),
String.valueOf(Objects.hash(nodeId)),
RemoteStoreUtils.invertLong(System.currentTimeMillis()),
String.valueOf(metadataVersion)
);
}

/**
* Initializes the cache which keeps track of all the segment files uploaded to the remote segment store.
* As this cache is specific to an instance of RemoteSegmentStoreDirectory, it is possible that cache becomes stale
Expand Down Expand Up @@ -317,28 +343,6 @@ static String getMetadataFilePrefixForCommit(long primaryTerm, long generation)
);
}

// Visible for testing
public static String getMetadataFilename(
long primaryTerm,
long generation,
long translogGeneration,
long uploadCounter,
int metadataVersion,
String nodeId
) {
return String.join(
SEPARATOR,
METADATA_PREFIX,
RemoteStoreUtils.invertLong(primaryTerm),
RemoteStoreUtils.invertLong(generation),
RemoteStoreUtils.invertLong(translogGeneration),
RemoteStoreUtils.invertLong(uploadCounter),
String.valueOf(Objects.hash(nodeId)),
RemoteStoreUtils.invertLong(System.currentTimeMillis()),
String.valueOf(metadataVersion)
);
}

// Visible for testing
static long getPrimaryTerm(String[] filenameTokens) {
return RemoteStoreUtils.invertLong(filenameTokens[1]);
Expand Down Expand Up @@ -595,10 +599,13 @@ public void uploadMetadata(
Directory storeDirectory,
long translogGeneration,
ReplicationCheckpoint replicationCheckpoint,
String nodeId
String nodeId,
boolean separateSegmentInfos
) throws IOException {
synchronized (this) {
String metadataFilename = MetadataFilenameUtils.getMetadataFilename(
String metadataFilename = getMetadataFilename(
MetadataFilenameUtils.SEPARATOR,
MetadataFilenameUtils.METADATA_PREFIX,
replicationCheckpoint.getPrimaryTerm(),
segmentInfosSnapshot.getGeneration(),
translogGeneration,
Expand All @@ -621,9 +628,49 @@ public void uploadMetadata(
}

ByteBuffersDataOutput byteBuffersIndexOutput = new ByteBuffersDataOutput();
segmentInfosSnapshot.write(
new ByteBuffersIndexOutput(byteBuffersIndexOutput, "Snapshot of SegmentInfos", "SegmentInfos")
);
if (separateSegmentInfos == false) {
segmentInfosSnapshot.write(
new ByteBuffersIndexOutput(byteBuffersIndexOutput, "Snapshot of SegmentInfos", "SegmentInfos")
);
} else {
String segmentInfoSnapshotFilename = getMetadataFilename(
MetadataFilenameUtils.SEPARATOR,
SEGMENT_INFOS_SNAPSHOT_PREFIX,
replicationCheckpoint.getPrimaryTerm(),
segmentInfosSnapshot.getGeneration(),
translogGeneration,
metadataUploadCounter.incrementAndGet(),
RemoteSegmentMetadata.CURRENT_VERSION,
nodeId
);
try {
try (
IndexOutput segmentInfosIndexOutput = storeDirectory.createOutput(
segmentInfoSnapshotFilename,
IOContext.DEFAULT
)
) {
segmentInfosSnapshot.write(segmentInfosIndexOutput);
}
remoteDataDirectory.copyFrom(
storeDirectory,
segmentInfoSnapshotFilename,
segmentInfoSnapshotFilename,
IOContext.DEFAULT
);
String segmentInfosSnapshotChecksum = getChecksumOfLocalFile(storeDirectory, segmentInfoSnapshotFilename);
UploadedSegmentMetadata segmentInfosSnapshotMetadata = new UploadedSegmentMetadata(
segmentInfoSnapshotFilename,
segmentInfoSnapshotFilename,
segmentInfosSnapshotChecksum,
storeDirectory.fileLength(segmentInfoSnapshotFilename)
);
segmentInfosSnapshotMetadata.setWrittenByMajor(segmentInfosSnapshot.getCommitLuceneVersion().major);
uploadedSegments.put(segmentInfoSnapshotFilename, segmentInfosSnapshotMetadata.toString());
} finally {
tryAndDeleteLocalFile(segmentInfoSnapshotFilename, storeDirectory);
}
}
byte[] segmentInfoSnapshotByteArray = byteBuffersIndexOutput.toArrayCopy();

metadataStreamWrapper.writeStream(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,9 +65,17 @@ public class RemoteStoreSettings {
Property.Dynamic
);

public static final Setting<Boolean> CLUSTER_REMOTE_SEGMENT_SEPARATE_METADATA_SEGMENTINFOS_SETTING = Setting.boolSetting(
"cluster.remote_store.segemnt.separate_metadata_segmentinfos",
false,
Property.NodeScope,
Property.Dynamic
);

private volatile TimeValue clusterRemoteTranslogBufferInterval;
private volatile int minRemoteSegmentMetadataFiles;
private volatile TimeValue clusterRemoteTranslogTransferTimeout;
private volatile Boolean clusterRemoteSegmentSeparateMetadataSegmentInfos;

public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) {
this.clusterRemoteTranslogBufferInterval = CLUSTER_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings);
Expand All @@ -87,6 +95,12 @@ public RemoteStoreSettings(Settings settings, ClusterSettings clusterSettings) {
CLUSTER_REMOTE_TRANSLOG_TRANSFER_TIMEOUT_SETTING,
this::setClusterRemoteTranslogTransferTimeout
);

this.clusterRemoteSegmentSeparateMetadataSegmentInfos = CLUSTER_REMOTE_SEGMENT_SEPARATE_METADATA_SEGMENTINFOS_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(
CLUSTER_REMOTE_SEGMENT_SEPARATE_METADATA_SEGMENTINFOS_SETTING,
this::setClusterRemoteSegmentSeparateMetadataSegmentInfos
);
}

public TimeValue getClusterRemoteTranslogBufferInterval() {
Expand All @@ -112,4 +126,12 @@ public TimeValue getClusterRemoteTranslogTransferTimeout() {
private void setClusterRemoteTranslogTransferTimeout(TimeValue clusterRemoteTranslogTransferTimeout) {
this.clusterRemoteTranslogTransferTimeout = clusterRemoteTranslogTransferTimeout;
}

public Boolean getClusterRemoteSegmentSeparateMetadataSegmentInfos() {
return clusterRemoteSegmentSeparateMetadataSegmentInfos;
}

public void setClusterRemoteSegmentSeparateMetadataSegmentInfos(Boolean clusterRemoteSegmentSeparateMetadataSegmentInfos) {
this.clusterRemoteSegmentSeparateMetadataSegmentInfos = clusterRemoteSegmentSeparateMetadataSegmentInfos;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@
import org.apache.lucene.index.IndexFormatTooOldException;
import org.apache.lucene.index.SegmentInfos;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.ChecksumIndexInput;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
import org.opensearch.OpenSearchCorruptionException;
Expand All @@ -25,6 +26,7 @@
import org.opensearch.core.action.ActionListener;
import org.opensearch.core.common.bytes.BytesReference;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.store.RemoteSegmentStoreDirectory;
import org.opensearch.index.store.Store;
import org.opensearch.index.store.StoreFileMetadata;
import org.opensearch.indices.recovery.MultiFileWriter;
Expand All @@ -36,6 +38,7 @@

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.Set;
Expand Down Expand Up @@ -283,19 +286,36 @@ private void updateFileRecoveryBytes(String fileName, long bytesRecovered) {
private void finalizeReplication(CheckpointInfoResponse checkpointInfoResponse) throws OpenSearchCorruptionException {
cancellableThreads.checkForCancel();
state.setStage(SegmentReplicationState.Stage.FINALIZE_REPLICATION);
byte[] segmentInfosBytes = checkpointInfoResponse.getInfosBytes();
// Handle empty SegmentInfos bytes for recovering replicas
if (checkpointInfoResponse.getInfosBytes() == null) {
if (segmentInfosBytes == null) {
return;
}
Store store = null;
try {
store = store();
store.incRef();
multiFileWriter.renameAllTempFiles();
final SegmentInfos infos = store.buildSegmentInfos(
checkpointInfoResponse.getInfosBytes(),
checkpointInfoResponse.getCheckpoint().getSegmentsGen()
);
final SegmentInfos infos;
if (segmentInfosBytes.length == 0) {
List<String> segmentInfosSnapshotFilenames = Arrays.stream(store.directory().listAll())
.filter(file -> file.startsWith(RemoteSegmentStoreDirectory.SEGMENT_INFOS_SNAPSHOT_PREFIX))
.collect(Collectors.toList());
assert segmentInfosSnapshotFilenames.size() == 1;
try (
ChecksumIndexInput segmentInfosInput = store.directory()
.openChecksumInput(segmentInfosSnapshotFilenames.get(0), IOContext.READ)
) {
infos = SegmentInfos.readCommit(
store.directory(),
segmentInfosInput,
checkpointInfoResponse.getCheckpoint().getSegmentsGen()
);
}
store.deleteQuiet(segmentInfosSnapshotFilenames.get(0));
} else {
infos = store.buildSegmentInfos(segmentInfosBytes, checkpointInfoResponse.getCheckpoint().getSegmentsGen());
}
indexShard.finalizeReplication(infos);
} catch (CorruptIndexException | IndexFormatTooNewException | IndexFormatTooOldException ex) {
// this is a fatal exception at this stage.
Expand Down
Loading
Loading