Skip to content

Commit

Permalink
Add async segment file download support from remote store (#9710)
Browse files Browse the repository at this point in the history
Signed-off-by: Kunal Kotwani <kkotwani@amazon.com>
(cherry picked from commit 9e90671)
Signed-off-by: github-actions[bot] <github-actions[bot]@users.noreply.github.com>
  • Loading branch information
github-actions[bot] committed Sep 22, 2023
1 parent c1c1cee commit c664261
Show file tree
Hide file tree
Showing 10 changed files with 274 additions and 31 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,7 @@ public RepositoryMetadata buildRepositoryMetadata(DiscoveryNode node, String nam
} else {
return super.buildRepositoryMetadata(node, name);
}

}

public void testRateLimitedRemoteUploads() throws Exception {
Expand Down
5 changes: 3 additions & 2 deletions server/src/main/java/org/opensearch/index/IndexService.java
Original file line number Diff line number Diff line change
Expand Up @@ -481,7 +481,7 @@ public synchronized IndexShard createShard(
Store remoteStore = null;
if (this.indexSettings.isRemoteStoreEnabled()) {
Directory remoteDirectory = remoteDirectoryFactory.newDirectory(this.indexSettings, path);
remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY);
remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path);
}

Directory directory = directoryFactory.newDirectory(this.indexSettings, path);
Expand All @@ -490,7 +490,8 @@ public synchronized IndexShard createShard(
this.indexSettings,
directory,
lock,
new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId))
new StoreCloseListener(shardId, () -> eventListener.onStoreClosed(shardId)),
path
);
eventListener.onStoreCreated(shardId);
indexShard = new IndexShard(
Expand Down
60 changes: 47 additions & 13 deletions server/src/main/java/org/opensearch/index/shard/IndexShard.java
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,8 @@
import org.opensearch.action.admin.indices.flush.FlushRequest;
import org.opensearch.action.admin.indices.forcemerge.ForceMergeRequest;
import org.opensearch.action.admin.indices.upgrade.post.UpgradeRequest;
import org.opensearch.action.support.GroupedActionListener;
import org.opensearch.action.support.PlainActionFuture;
import org.opensearch.action.support.replication.PendingReplicationActions;
import org.opensearch.action.support.replication.ReplicationResponse;
import org.opensearch.cluster.metadata.DataStream;
Expand Down Expand Up @@ -97,7 +99,6 @@
import org.opensearch.common.util.concurrent.RunOnce;
import org.opensearch.common.util.concurrent.ThreadContext;
import org.opensearch.common.util.io.IOUtils;
import org.opensearch.common.util.set.Sets;
import org.opensearch.core.Assertions;
import org.opensearch.core.action.ActionListener;
import org.opensearch.core.common.unit.ByteSizeValue;
Expand Down Expand Up @@ -200,6 +201,7 @@
import java.nio.channels.FileChannel;
import java.nio.charset.StandardCharsets;
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
Expand Down Expand Up @@ -859,7 +861,7 @@ public void relocated(
synchronized (mutex) {
verifyRelocatingState();
replicationTracker.completeRelocationHandoff(); // make changes to primaryMode and relocated flag only under
// mutex
// mutex
}
} catch (final Exception e) {
try {
Expand Down Expand Up @@ -3801,7 +3803,7 @@ private EngineConfig newEngineConfig(LongSupplier globalCheckpointSupplier) thro
replicationTracker::isPrimaryMode,
translogFactorySupplier.apply(indexSettings, shardRouting),
isTimeSeriesDescSortOptimizationEnabled() ? DataStream.TIMESERIES_LEAF_SORTER : null // DESC @timestamp default order for
// timeseries
// timeseries
);
}

Expand Down Expand Up @@ -4862,39 +4864,71 @@ private String copySegmentFiles(
Map<String, RemoteSegmentStoreDirectory.UploadedSegmentMetadata> uploadedSegments,
boolean overrideLocal
) throws IOException {
List<String> downloadedSegments = new ArrayList<>();
List<String> skippedSegments = new ArrayList<>();
Set<String> toDownloadSegments = new HashSet<>();
Set<String> skippedSegments = new HashSet<>();
String segmentNFile = null;

try {
Set<String> localSegmentFiles = Sets.newHashSet(storeDirectory.listAll());
if (overrideLocal) {
for (String file : localSegmentFiles) {
for (String file : storeDirectory.listAll()) {
storeDirectory.deleteFile(file);
}
}

for (String file : uploadedSegments.keySet()) {
long checksum = Long.parseLong(uploadedSegments.get(file).getChecksum());
if (overrideLocal || localDirectoryContains(storeDirectory, file, checksum) == false) {
storeDirectory.copyFrom(sourceRemoteDirectory, file, file, IOContext.DEFAULT);
downloadedSegments.add(file);
toDownloadSegments.add(file);
} else {
skippedSegments.add(file);
}
if (targetRemoteDirectory != null) {
targetRemoteDirectory.copyFrom(storeDirectory, file, file, IOContext.DEFAULT);
}

if (file.startsWith(IndexFileNames.SEGMENTS)) {
assert segmentNFile == null : "There should be only one SegmentInfosSnapshot file";
segmentNFile = file;
}
}

if (toDownloadSegments.isEmpty() == false) {
try {
final PlainActionFuture<Void> completionListener = PlainActionFuture.newFuture();
downloadSegments(storeDirectory, sourceRemoteDirectory, targetRemoteDirectory, toDownloadSegments, completionListener);
completionListener.actionGet();
} catch (Exception e) {
throw new IOException("Error occurred when downloading segments from remote store", e);

Check warning on line 4898 in server/src/main/java/org/opensearch/index/shard/IndexShard.java

View check run for this annotation

Codecov / codecov/patch

server/src/main/java/org/opensearch/index/shard/IndexShard.java#L4897-L4898

Added lines #L4897 - L4898 were not covered by tests
}
}
} finally {
logger.trace("Downloaded segments here: {}", downloadedSegments);
logger.trace("Downloaded segments here: {}", toDownloadSegments);
logger.trace("Skipped download for segments here: {}", skippedSegments);
}

return segmentNFile;
}

private void downloadSegments(
Directory storeDirectory,
RemoteSegmentStoreDirectory sourceRemoteDirectory,
RemoteSegmentStoreDirectory targetRemoteDirectory,
Set<String> toDownloadSegments,
ActionListener<Void> completionListener
) {
final Path indexPath = store.shardPath() == null ? null : store.shardPath().resolveIndex();
final GroupedActionListener<Void> batchDownloadListener = new GroupedActionListener<>(
ActionListener.map(completionListener, v -> null),
toDownloadSegments.size()
);

final ActionListener<String> segmentsDownloadListener = ActionListener.map(batchDownloadListener, fileName -> {
if (targetRemoteDirectory != null) {
targetRemoteDirectory.copyFrom(storeDirectory, fileName, fileName, IOContext.DEFAULT);
}
return null;
});

toDownloadSegments.forEach(file -> sourceRemoteDirectory.copyTo(file, storeDirectory, indexPath, segmentsDownloadListener));
}

private boolean localDirectoryContains(Directory localDirectory, String file, long checksum) {
try (IndexInput indexInput = localDirectory.openInput(file, IOContext.DEFAULT)) {
if (checksum == CodecUtil.retrieveChecksum(indexInput)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Version;
import org.opensearch.common.UUIDs;
import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer;
import org.opensearch.common.io.VersionedCodecStreamWrapper;
import org.opensearch.common.lucene.store.ByteArrayIndexInput;
import org.opensearch.core.action.ActionListener;
Expand All @@ -40,6 +41,7 @@
import java.io.IOException;
import java.io.InputStream;
import java.nio.file.NoSuchFileException;
import java.nio.file.Path;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
Expand Down Expand Up @@ -434,6 +436,41 @@ public void copyFrom(Directory from, String src, IOContext context, ActionListen
}
}

/**
* Copies an existing {@code source} file from this directory to a non-existent file (also
* named {@code source}) in either {@code destinationDirectory} or {@code destinationPath}.
* If the blob container backing this directory supports multipart downloads, the {@code source}
* file will be downloaded (potentially in multiple concurrent parts) directly to
* {@code destinationPath}. This method will return immediately and {@code fileCompletionListener}
* will be notified upon completion.
* <p>
* If multipart downloads are not supported, then {@code source} file will be copied to a file named
* {@code source} in a single part to {@code destinationDirectory}. The download will happen on the
* calling thread and {@code fileCompletionListener} will be notified synchronously before this
* method returns.
*
* @param source The source file name
* @param destinationDirectory The destination directory (if multipart is not supported)
* @param destinationPath The destination path (if multipart is supported)
* @param fileCompletionListener The listener to notify of completion
*/
public void copyTo(String source, Directory destinationDirectory, Path destinationPath, ActionListener<String> fileCompletionListener) {
final String blobName = getExistingRemoteFilename(source);
if (destinationPath != null && remoteDataDirectory.getBlobContainer() instanceof AsyncMultiStreamBlobContainer) {
final AsyncMultiStreamBlobContainer blobContainer = (AsyncMultiStreamBlobContainer) remoteDataDirectory.getBlobContainer();
final Path destinationFilePath = destinationPath.resolve(source);
blobContainer.asyncBlobDownload(blobName, destinationFilePath, threadPool, fileCompletionListener);
} else {
// Fallback to older mechanism of downloading the file
try {
destinationDirectory.copyFrom(this, source, source, IOContext.DEFAULT);
fileCompletionListener.onResponse(source);
} catch (IOException e) {
fileCompletionListener.onFailure(e);
}
}
}

/**
* This acquires a lock on a given commit by creating a lock file in lock directory using {@code FileLockInfo}
*
Expand Down
20 changes: 18 additions & 2 deletions server/src/main/java/org/opensearch/index/store/Store.java
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,7 @@
import org.apache.lucene.util.Version;
import org.opensearch.ExceptionsHelper;
import org.opensearch.common.UUIDs;
import org.opensearch.common.annotation.InternalApi;
import org.opensearch.common.io.stream.BytesStreamOutput;
import org.opensearch.common.logging.Loggers;
import org.opensearch.common.lucene.Lucene;
Expand All @@ -92,6 +93,7 @@
import org.opensearch.index.seqno.SequenceNumbers;
import org.opensearch.index.shard.AbstractIndexShardComponent;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.ShardPath;
import org.opensearch.index.translog.Translog;

import java.io.Closeable;
Expand Down Expand Up @@ -179,6 +181,7 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref
private final ReentrantReadWriteLock metadataLock = new ReentrantReadWriteLock();
private final ShardLock shardLock;
private final OnClose onClose;
private final ShardPath shardPath;

// used to ref count files when a new Reader is opened for PIT/Scroll queries
// prevents segment files deletion until the PIT/Scroll expires or is discarded
Expand All @@ -192,17 +195,25 @@ protected void closeInternal() {
};

public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock) {
this(shardId, indexSettings, directory, shardLock, OnClose.EMPTY);
this(shardId, indexSettings, directory, shardLock, OnClose.EMPTY, null);
}

public Store(ShardId shardId, IndexSettings indexSettings, Directory directory, ShardLock shardLock, OnClose onClose) {
public Store(
ShardId shardId,
IndexSettings indexSettings,
Directory directory,
ShardLock shardLock,
OnClose onClose,
ShardPath shardPath
) {
super(shardId, indexSettings);
final TimeValue refreshInterval = indexSettings.getValue(INDEX_STORE_STATS_REFRESH_INTERVAL_SETTING);
logger.debug("store stats are refreshed with refresh_interval [{}]", refreshInterval);
ByteSizeCachingDirectory sizeCachingDir = new ByteSizeCachingDirectory(directory, refreshInterval);
this.directory = new StoreDirectory(sizeCachingDir, Loggers.getLogger("index.store.deletes", shardId));
this.shardLock = shardLock;
this.onClose = onClose;
this.shardPath = shardPath;
assert onClose != null;
assert shardLock != null;
assert shardLock.getShardId().equals(shardId);
Expand All @@ -213,6 +224,11 @@ public Directory directory() {
return directory;
}

@InternalApi
public ShardPath shardPath() {
return shardPath;
}

/**
* Returns the last committed segments info for this store
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,18 +13,20 @@
import org.apache.lucene.index.SegmentInfos;
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.support.GroupedActionListener;
import org.opensearch.common.concurrent.GatedCloseable;
import org.opensearch.core.action.ActionListener;
import org.opensearch.index.shard.IndexShard;
import org.opensearch.index.shard.IndexShardState;
import org.opensearch.index.shard.ShardPath;
import org.opensearch.index.store.RemoteSegmentStoreDirectory;
import org.opensearch.index.store.Store;
import org.opensearch.index.store.StoreFileMetadata;
import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata;
import org.opensearch.indices.replication.checkpoint.ReplicationCheckpoint;

import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
Expand Down Expand Up @@ -106,31 +108,49 @@ public void getSegmentFiles(
logger.trace("Downloading segments files from remote store {}", filesToFetch);

RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile();
List<StoreFileMetadata> downloadedSegments = new ArrayList<>();
List<StoreFileMetadata> toDownloadSegments = new ArrayList<>();
Collection<String> directoryFiles = List.of(indexShard.store().directory().listAll());
if (remoteSegmentMetadata != null) {
try {
indexShard.store().incRef();
indexShard.remoteStore().incRef();
final Directory storeDirectory = indexShard.store().directory();
final ShardPath shardPath = indexShard.shardPath();
for (StoreFileMetadata fileMetadata : filesToFetch) {
String file = fileMetadata.name();
assert directoryFiles.contains(file) == false : "Local store already contains the file " + file;
storeDirectory.copyFrom(remoteDirectory, file, file, IOContext.DEFAULT);
downloadedSegments.add(fileMetadata);
toDownloadSegments.add(fileMetadata);
}
logger.trace("Downloaded segments from remote store {}", downloadedSegments);
downloadSegments(storeDirectory, remoteDirectory, toDownloadSegments, shardPath, listener);
logger.trace("Downloaded segments from remote store {}", toDownloadSegments);
} finally {
indexShard.store().decRef();
indexShard.remoteStore().decRef();
}
}
listener.onResponse(new GetSegmentFilesResponse(downloadedSegments));
} catch (Exception e) {
listener.onFailure(e);
}
}

private void downloadSegments(
Directory storeDirectory,
RemoteSegmentStoreDirectory remoteStoreDirectory,
List<StoreFileMetadata> toDownloadSegments,
ShardPath shardPath,
ActionListener<GetSegmentFilesResponse> completionListener
) {
final Path indexPath = shardPath == null ? null : shardPath.resolveIndex();
final GroupedActionListener<Void> batchDownloadListener = new GroupedActionListener<>(
ActionListener.map(completionListener, v -> new GetSegmentFilesResponse(toDownloadSegments)),
toDownloadSegments.size()
);
ActionListener<String> segmentsDownloadListener = ActionListener.map(batchDownloadListener, result -> null);
toDownloadSegments.forEach(
fileMetadata -> remoteStoreDirectory.copyTo(fileMetadata.name(), storeDirectory, indexPath, segmentsDownloadListener)
);
}

@Override
public String getDescription() {
return "RemoteStoreReplicationSource";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1718,7 +1718,7 @@ public Set<String> getPendingDeletions() throws IOException {
}
};

try (Store store = createStore(shardId, new IndexSettings(metadata, Settings.EMPTY), directory)) {
try (Store store = createStore(shardId, new IndexSettings(metadata, Settings.EMPTY), directory, shardPath)) {
IndexShard shard = newShard(
shardRouting,
shardPath,
Expand Down
Loading

0 comments on commit c664261

Please sign in to comment.