Skip to content

Commit

Permalink
[Backport 2.x] [Remote Segment Store] Make metadata file immutable (#…
Browse files Browse the repository at this point in the history
…8363) (#8545)

Signed-off-by: Sachin Kale <[email protected]>
  • Loading branch information
sachinpkale authored Jul 11, 2023
1 parent b40d956 commit 3858cad
Show file tree
Hide file tree
Showing 12 changed files with 358 additions and 289 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -857,6 +857,7 @@ public void testReplicaHasDiffFilesThanPrimary() throws Exception {
public void testPressureServiceStats() throws Exception {
final String primaryNode = internalCluster().startDataOnlyNode();
createIndex(INDEX_NAME);
ensureYellow(INDEX_NAME);
final String replicaNode = internalCluster().startDataOnlyNode();
ensureGreen(INDEX_NAME);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -316,6 +316,6 @@ public void testStaleCommitDeletionWithoutInvokeFlush() throws Exception {
.get()
.getSetting(INDEX_NAME, IndexMetadata.SETTING_INDEX_UUID);
Path indexPath = Path.of(String.valueOf(absolutePath), indexUUID, "/0/segments/metadata");
assertEquals(1, getFileCount(indexPath));
assertEquals(numberOfIterations, getFileCount(indexPath));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import org.opensearch.index.seqno.SequenceNumbers;
import org.opensearch.index.store.RemoteSegmentStoreDirectory;
import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata;
import org.opensearch.index.translog.Translog;
import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint;
import org.opensearch.indices.replication.checkpoint.SegmentReplicationCheckpointPublisher;
import org.opensearch.threadpool.Scheduler;
Expand Down Expand Up @@ -183,6 +184,11 @@ public void afterRefresh(boolean didRefresh) {

private synchronized void syncSegments(boolean isRetry) {
if (indexShard.getReplicationTracker().isPrimaryMode() == false) {
logger.info("syncSegments is only supported with primaryMode=true, current value is false. Skipping");
return;
}
if (indexShard.getEngine() instanceof InternalEngine == false) {
logger.info("syncSegments is only supported for InternalEngine, called with {}. Skipping", indexShard.getEngine());
return;
}
beforeSegmentsSync(isRetry);
Expand Down Expand Up @@ -345,19 +351,28 @@ private boolean isRefreshAfterCommit() throws IOException {
}

void uploadMetadata(Collection<String> localSegmentsPostRefresh, SegmentInfos segmentInfos) throws IOException {

final long maxSeqNo = ((InternalEngine) indexShard.getEngine()).currentOngoingRefreshCheckpoint();
SegmentInfos segmentInfosSnapshot = segmentInfos.clone();
Map<String, String> userData = segmentInfosSnapshot.getUserData();
userData.put(LOCAL_CHECKPOINT_KEY, String.valueOf(maxSeqNo));
userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo));
segmentInfosSnapshot.setUserData(userData, false);

remoteDirectory.uploadMetadata(
localSegmentsPostRefresh,
segmentInfosSnapshot,
storeDirectory,
indexShard.getOperationPrimaryTerm()
);
Translog.TranslogGeneration translogGeneration = ((InternalEngine) indexShard.getEngine()).translogManager()
.getTranslogGeneration();
if (translogGeneration == null) {
throw new UnsupportedOperationException("Encountered null TranslogGeneration while uploading metadata to remote segment store");
} else {
long translogFileGeneration = translogGeneration.translogFileGeneration;
remoteDirectory.uploadMetadata(
localSegmentsPostRefresh,
segmentInfosSnapshot,
storeDirectory,
indexShard.getOperationPrimaryTerm(),
translogFileGeneration
);
}
}

private boolean uploadNewSegments(Collection<String> localSegmentsPostRefresh) throws IOException {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,17 +13,24 @@
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.store.Lock;
import org.opensearch.action.ActionListener;
import org.opensearch.action.LatchedActionListener;
import org.opensearch.common.blobstore.BlobContainer;
import org.opensearch.common.blobstore.BlobMetadata;

import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.NoSuchFileException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;

/**
* A {@code RemoteDirectory} provides an abstraction layer for storing a list of files to a remote store.
Expand Down Expand Up @@ -61,6 +68,40 @@ public Collection<String> listFilesByPrefix(String filenamePrefix) throws IOExce
return blobContainer.listBlobsByPrefix(filenamePrefix).keySet();
}

public List<String> listFilesByPrefixInLexicographicOrder(String filenamePrefix, int limit) throws IOException {
List<String> sortedBlobList = new ArrayList<>();
AtomicReference<Exception> exception = new AtomicReference<>();
final CountDownLatch latch = new CountDownLatch(1);
LatchedActionListener<List<BlobMetadata>> actionListener = new LatchedActionListener<>(new ActionListener<>() {
@Override
public void onResponse(List<BlobMetadata> blobMetadata) {
sortedBlobList.addAll(blobMetadata.stream().map(BlobMetadata::name).collect(Collectors.toList()));
}

@Override
public void onFailure(Exception e) {
exception.set(e);
}
}, latch);

try {
blobContainer.listBlobsByPrefixInSortedOrder(
filenamePrefix,
limit,
BlobContainer.BlobNameSortOrder.LEXICOGRAPHIC,
actionListener
);
latch.await();
} catch (InterruptedException e) {
throw new IOException("Exception in listFilesByPrefixInLexicographicOrder with prefix: " + filenamePrefix, e);
}
if (exception.get() != null) {
throw new IOException(exception.get());
} else {
return sortedBlobList;
}
}

/**
* Removes an existing file in the directory.
*
Expand Down
Loading

0 comments on commit 3858cad

Please sign in to comment.