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 Segment Store] Make metadata file immutable #8363

Merged
merged 12 commits into from
Jul 6, 2023
Merged
Show file tree
Hide file tree
Changes from 11 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
Original file line number Diff line number Diff line change
Expand Up @@ -800,6 +800,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 @@ -320,6 +320,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 @@ -349,11 +349,13 @@ void uploadMetadata(Collection<String> localSegmentsPostRefresh, SegmentInfos se
userData.put(SequenceNumbers.MAX_SEQ_NO, Long.toString(maxSeqNo));
segmentInfosSnapshot.setUserData(userData, false);

long translogGeneration = indexShard.getEngine().translogManager().getTranslogGeneration().translogFileGeneration;
gbbafna marked this conversation as resolved.
Show resolved Hide resolved
remoteDirectory.uploadMetadata(
localSegmentsPostRefresh,
segmentInfosSnapshot,
storeDirectory,
indexShard.getOperationPrimaryTerm()
indexShard.getOperationPrimaryTerm(),
translogGeneration
gbbafna marked this conversation as resolved.
Show resolved Hide resolved
);
}

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