From 3c6ad602a564935ed301f7f7a3967a228654ed51 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Wed, 13 Mar 2024 16:26:02 +0530 Subject: [PATCH 01/22] Composite Directory POC Signed-off-by: Shreyansh Ray --- .../remotestore/CompositeDirectoryIT.java | 66 ++++++++ .../metadata/MetadataCreateIndexService.java | 12 ++ .../org/opensearch/index/IndexModule.java | 7 +- .../shard/RemoteStoreRefreshListener.java | 5 + .../index/store/CompositeDirectory.java | 152 ++++++++++++++++++ .../store/CompositeDirectoryFactory.java | 52 ++++++ .../CompositeDirectoryTransferManager.java | 81 ++++++++++ .../index/store/TransferManager.java | 32 ++++ .../remote/utils/filetracker/FileState.java | 15 ++ .../utils/filetracker/FileTrackingInfo.java | 27 ++++ .../remote/utils/filetracker/FileType.java | 18 +++ .../main/java/org/opensearch/node/Node.java | 8 +- 12 files changed, 472 insertions(+), 3 deletions(-) create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java create mode 100644 server/src/main/java/org/opensearch/index/store/CompositeDirectory.java create mode 100644 server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java create mode 100644 server/src/main/java/org/opensearch/index/store/CompositeDirectoryTransferManager.java create mode 100644 server/src/main/java/org/opensearch/index/store/TransferManager.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java new file mode 100644 index 0000000000000..abb291644e479 --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java @@ -0,0 +1,66 @@ +/* + * 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.remotestore; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; +import org.opensearch.action.admin.indices.get.GetIndexRequest; +import org.opensearch.action.admin.indices.get.GetIndexResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.index.IndexModule; +import org.opensearch.index.IndexService; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.CompositeDirectory; +import org.opensearch.indices.IndicesService; +import org.opensearch.indices.replication.common.ReplicationType; + +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; +import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; + +public class CompositeDirectoryIT extends RemoteStoreBaseIntegTestCase { + public void testCompositeDirectory() throws Exception { + Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), "compositefs") + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .build(); + assertAcked(client().admin().indices().prepareCreate("test-idx-1").setSettings(settings).get()); + GetIndexResponse getIndexResponse = client().admin() + .indices() + .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) + .get(); + boolean indexServiceFound = false; + String[] nodes = internalCluster().getNodeNames(); + for (String node : nodes) { + IndexService indexService = internalCluster().getInstance(IndicesService.class, node).indexService(resolveIndex("test-idx-1")); + if (indexService == null) { + continue; + } + IndexShard shard = indexService.getShardOrNull(0); + Directory directory = (((FilterDirectory) (((FilterDirectory) (shard.store().directory())).getDelegate())).getDelegate()); + assertTrue(directory instanceof CompositeDirectory); + indexServiceFound = true; + } + assertTrue(indexServiceFound); + Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); + assertEquals(ReplicationType.SEGMENT.toString(), indexSettings.get(SETTING_REPLICATION_TYPE)); + assertEquals("true", indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); + assertEquals(REPOSITORY_NAME, indexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); + assertEquals(REPOSITORY_2_NAME, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); + assertEquals("compositefs", indexSettings.get("index.store.type")); + + ensureGreen("test-idx-1"); + indexData(10, false, "test-idx-1"); + ensureGreen("test-idx-1"); + } +} diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 16edec112f123..7b21e39026c26 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -75,6 +75,7 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.Strings; @@ -995,6 +996,7 @@ static Settings aggregateIndexSettings( validateStoreTypeSettings(indexSettings); validateRefreshIntervalSettings(request.settings(), clusterSettings); validateTranslogDurabilitySettings(request.settings(), clusterSettings, settings); + validateCompositeFS(request.settings()); return indexSettings; } @@ -1688,4 +1690,14 @@ static void validateTranslogDurabilitySettings(Settings requestSettings, Cluster } } + + public static void validateCompositeFS(Settings indexSettings) { + if (indexSettings.get(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), "") + .equalsIgnoreCase(IndexModule.Type.COMPOSITEFS.getSettingsKey()) + && !FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING)) { + throw new IllegalArgumentException( + "ERROR - Composite FS store type can be enabled only if Feature Flag for Writable Remote Index is true" + ); + } + } } diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 3c4cb4fd596c1..16dce2645b2d3 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -74,6 +74,7 @@ import org.opensearch.index.shard.IndexingOperationListener; import org.opensearch.index.shard.SearchOperationListener; import org.opensearch.index.similarity.SimilarityService; +import org.opensearch.index.store.CompositeDirectoryFactory; import org.opensearch.index.store.FsDirectoryFactory; import org.opensearch.index.store.remote.directory.RemoteSnapshotDirectoryFactory; import org.opensearch.index.store.remote.filecache.FileCache; @@ -506,7 +507,8 @@ public enum Type { MMAPFS("mmapfs"), SIMPLEFS("simplefs"), FS("fs"), - REMOTE_SNAPSHOT("remote_snapshot"); + REMOTE_SNAPSHOT("remote_snapshot"), + COMPOSITEFS("compositefs"); private final String settingsKey; private final boolean deprecated; @@ -788,6 +790,9 @@ public static Map createBuiltInDirect new RemoteSnapshotDirectoryFactory(repositoriesService, threadPool, remoteStoreFileCache) ); break; + case COMPOSITEFS: + factories.put(type.getSettingsKey(), new CompositeDirectoryFactory(repositoriesService, remoteStoreFileCache)); + break; default: throw new IllegalStateException("No directory factory mapping for built-in type " + type); } 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 8773b37aa7d4c..f3c2d27c20b58 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -30,6 +30,7 @@ import org.opensearch.index.engine.InternalEngine; import org.opensearch.index.remote.RemoteSegmentTransferTracker; import org.opensearch.index.seqno.SequenceNumbers; +import org.opensearch.index.store.CompositeDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.translog.Translog; @@ -298,6 +299,10 @@ public void onFailure(Exception e) { // Start the segments files upload uploadNewSegments(localSegmentsPostRefresh, localSegmentsSizeMap, segmentUploadsCompletedListener); + Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); + if (directory instanceof CompositeDirectory) { + ((CompositeDirectory) directory).afterSyncToRemote(localSegmentsPostRefresh); + } if (latch.await( remoteStoreSettings.getClusterRemoteSegmentTransferTimeout().millis(), TimeUnit.MILLISECONDS diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java new file mode 100644 index 0000000000000..080b42a2b65a2 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -0,0 +1,152 @@ +/* + * 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.store; + +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.FilterDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.store.Lock; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.index.store.remote.filecache.CachedIndexInput; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.utils.filetracker.FileState; +import org.opensearch.index.store.remote.utils.filetracker.FileType; + +import java.io.IOException; +import java.util.Collection; +import java.util.Set; + +public class CompositeDirectory extends FilterDirectory { + + private final FSDirectory localDirectory; + private final TransferManager transferManager; + private final FileCache fileCache; + + public CompositeDirectory(FSDirectory localDirectory, BlobContainer blobContainer, FileCache fileCache) { + super(localDirectory); + this.localDirectory = localDirectory; + this.fileCache = fileCache; + this.transferManager = new CompositeDirectoryTransferManager(fileCache, blobContainer); + } + + @Override + public String[] listAll() throws IOException { + return localDirectory.listAll(); + } + + @Override + public void deleteFile(String name) throws IOException { + super.deleteFile(name); + transferManager.removeFileFromTracker(name); + fileCache.remove(localDirectory.getDirectory().resolve(name)); + } + + @Override + public long fileLength(String name) throws IOException { + return localDirectory.fileLength(name); + } + + @Override + public IndexOutput createOutput(String name, IOContext context) throws IOException { + transferManager.trackFile(name, FileState.DISK, FileType.NON_BLOCK); + return localDirectory.createOutput(name, context); + } + + @Override + public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException { + return localDirectory.createTempOutput(prefix, suffix, context); + } + + @Override + public void sync(Collection names) throws IOException { + localDirectory.sync(names); + } + + @Override + public void syncMetaData() throws IOException { + localDirectory.syncMetaData(); + } + + @Override + public void rename(String source, String dest) throws IOException { + localDirectory.rename(source, dest); + transferManager.trackFile(dest, transferManager.getFileState(source), transferManager.getFileType(source)); + transferManager.removeFileFromTracker(source); + } + + @Override + public IndexInput openInput(String name, IOContext context) throws IOException { + if (!transferManager.isFilePresent(name)) { + return localDirectory.openInput(name, context); + } + IndexInput indexInput = null; + switch (transferManager.getFileState(name)) { + case DISK: + indexInput = localDirectory.openInput(name, context); + break; + + case CACHE: + indexInput = fileCache.get(localDirectory.getDirectory().resolve(name)).getIndexInput(); + break; + + case REMOTE_ONLY: + // TODO - return an implementation of OnDemandBlockIndexInput where the fetchBlock method is implemented + break; + } + return indexInput; + } + + @Override + public Lock obtainLock(String name) throws IOException { + return localDirectory.obtainLock(name); + } + + @Override + public void close() throws IOException { + localDirectory.close(); + } + + @Override + public Set getPendingDeletions() throws IOException { + return localDirectory.getPendingDeletions(); + } + + public void afterSyncToRemote(Collection files) throws IOException { + for (String fileName : files) { + if (transferManager.isFilePresent(fileName) && !transferManager.getFileState(fileName).equals(FileState.CACHE)) { + transferManager.updateFileState(fileName, FileState.CACHE); + } + fileCache.put(localDirectory.getDirectory().resolve(fileName), new CachedIndexInput() { + @Override + public IndexInput getIndexInput() throws IOException { + return localDirectory.openInput(fileName, IOContext.READ); + } + + @Override + public long length() { + try { + return localDirectory.fileLength(fileName); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean isClosed() { + return false; + } + + @Override + public void close() {} + }); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java new file mode 100644 index 0000000000000..52095614f56a9 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java @@ -0,0 +1,52 @@ +/* + * 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.store; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.common.blobstore.BlobPath; +import org.opensearch.index.IndexSettings; +import org.opensearch.index.shard.ShardPath; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.plugins.IndexStorePlugin; +import org.opensearch.repositories.RepositoriesService; +import org.opensearch.repositories.Repository; +import org.opensearch.repositories.blobstore.BlobStoreRepository; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.function.Supplier; + +public class CompositeDirectoryFactory implements IndexStorePlugin.DirectoryFactory { + + private final Supplier repositoriesService; + private final FileCache remoteStoreFileCache; + + public CompositeDirectoryFactory(Supplier repositoriesService, FileCache remoteStoreFileCache) { + this.repositoriesService = repositoriesService; + this.remoteStoreFileCache = remoteStoreFileCache; + } + + @Override + public Directory newDirectory(IndexSettings indexSettings, ShardPath shardPath) throws IOException { + String repositoryName = indexSettings.getRemoteStoreRepository(); + Repository repository = repositoriesService.get().repository(repositoryName); + BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; + String shardId = String.valueOf(shardPath.getShardId().getId()); + String indexUUID = indexSettings.getIndex().getUUID(); + BlobPath blobPath = blobStoreRepository.basePath().add(indexUUID).add(shardId).add("segments").add("data"); + final BlobContainer blobContainer = blobStoreRepository.blobStore().blobContainer(blobPath); + + final Path location = shardPath.resolveIndex(); + final FSDirectory primaryDirectory = FSDirectory.open(location); + + return new CompositeDirectory(primaryDirectory, blobContainer, remoteStoreFileCache); + } +} diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryTransferManager.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryTransferManager.java new file mode 100644 index 0000000000000..4e10a6db80db3 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryTransferManager.java @@ -0,0 +1,81 @@ +/* + * 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.store; + +import org.apache.lucene.store.IndexInput; +import org.opensearch.common.blobstore.BlobContainer; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.utils.BlobFetchRequest; +import org.opensearch.index.store.remote.utils.filetracker.FileState; +import org.opensearch.index.store.remote.utils.filetracker.FileTrackingInfo; +import org.opensearch.index.store.remote.utils.filetracker.FileType; + +import java.util.HashMap; +import java.util.Map; + +public class CompositeDirectoryTransferManager implements TransferManager { + + private FileCache fileCache; + private Map fileTracker; + private BlobContainer blobContainer; + + public CompositeDirectoryTransferManager(FileCache fileCache, BlobContainer blobContainer) { + this.fileCache = fileCache; + this.blobContainer = blobContainer; + this.fileTracker = new HashMap<>(); + } + + @Override + public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) { + // TODO - This function will fetch the requested data from blobContainer + return null; + } + + public void trackFile(String name, FileState fileState, FileType fileType) { + if (!fileTracker.containsKey(name)) { + fileTracker.put(name, new FileTrackingInfo(fileState, fileType)); + } + } + + public void updateFileType(String name, FileType fileType) { + FileTrackingInfo fileTrackingInfo = fileTracker.get(name); + if (fileTrackingInfo != null) { + fileTracker.put(name, new FileTrackingInfo(fileTrackingInfo.getFileState(), fileType)); + } + } + + public void updateFileState(String name, FileState fileState) { + FileTrackingInfo fileTrackingInfo = fileTracker.get(name); + if (fileTrackingInfo != null) { + fileTracker.put(name, new FileTrackingInfo(fileState, fileTrackingInfo.getFileType())); + } + } + + public void removeFileFromTracker(String name) { + fileTracker.remove(name); + } + + public FileState getFileState(String name) { + if (!fileTracker.containsKey(name)) { + return null; + } + return fileTracker.get(name).getFileState(); + } + + public FileType getFileType(String name) { + if (!fileTracker.containsKey(name)) { + return null; + } + return fileTracker.get(name).getFileType(); + } + + public boolean isFilePresent(String name) { + return fileTracker.containsKey(name); + } +} diff --git a/server/src/main/java/org/opensearch/index/store/TransferManager.java b/server/src/main/java/org/opensearch/index/store/TransferManager.java new file mode 100644 index 0000000000000..5029c51a34e34 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/TransferManager.java @@ -0,0 +1,32 @@ +/* + * 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.store; + +import org.apache.lucene.store.IndexInput; +import org.opensearch.index.store.remote.utils.BlobFetchRequest; +import org.opensearch.index.store.remote.utils.filetracker.FileState; +import org.opensearch.index.store.remote.utils.filetracker.FileType; + +public interface TransferManager { + IndexInput fetchBlob(BlobFetchRequest blobFetchRequest); + + void trackFile(String name, FileState fileState, FileType fileType); + + void updateFileType(String name, FileType fileType); + + void updateFileState(String name, FileState fileState); + + void removeFileFromTracker(String name); + + FileState getFileState(String name); + + FileType getFileType(String name); + + boolean isFilePresent(String name); +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java new file mode 100644 index 0000000000000..564bb87e03857 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java @@ -0,0 +1,15 @@ +/* + * 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.store.remote.utils.filetracker; + +public enum FileState { + DISK, + CACHE, + REMOTE_ONLY; +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java new file mode 100644 index 0000000000000..60310b0fc39a2 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java @@ -0,0 +1,27 @@ +/* + * 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.store.remote.utils.filetracker; + +public class FileTrackingInfo { + private final FileState fileState; + private final FileType fileType; + + public FileTrackingInfo(FileState fileState, FileType fileType) { + this.fileState = fileState; + this.fileType = fileType; + } + + public FileState getFileState() { + return fileState; + } + + public FileType getFileType() { + return fileType; + } +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java new file mode 100644 index 0000000000000..18506f7062b7c --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java @@ -0,0 +1,18 @@ +/* + * 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.store.remote.utils.filetracker; + +public enum FileType { + BLOCK, + NON_BLOCK; + + public boolean isBlockFile(FileType fileType) { + return fileType.equals(FileType.BLOCK); + } +} diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index 397949525a3ec..b24d342fb616b 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -2004,7 +2004,8 @@ DiscoveryNode getNode() { * Else it configures the size to 80% of available capacity for a dedicated search node, if not explicitly defined. */ private void initializeFileCache(Settings settings, CircuitBreaker circuitBreaker) throws IOException { - if (DiscoveryNode.isSearchNode(settings)) { + boolean isWritableRemoteIndexEnabled = FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING); + if (DiscoveryNode.isSearchNode(settings) || isWritableRemoteIndexEnabled) { NodeEnvironment.NodePath fileCacheNodePath = nodeEnvironment.fileCacheNodePath(); long capacity = NODE_SEARCH_CACHE_SIZE_SETTING.get(settings).getBytes(); FsInfo.Path info = ExceptionsHelper.catchAsRuntimeException(() -> FsProbe.getFSInfo(fileCacheNodePath)); @@ -2013,7 +2014,10 @@ private void initializeFileCache(Settings settings, CircuitBreaker circuitBreake // Initialize default values for cache if NODE_SEARCH_CACHE_SIZE_SETTING is not set. if (capacity == 0) { // If node is not a dedicated search node without configuration, prevent cache initialization - if (DiscoveryNode.getRolesFromSettings(settings).stream().anyMatch(role -> !DiscoveryNodeRole.SEARCH_ROLE.equals(role))) { + if (!isWritableRemoteIndexEnabled + && DiscoveryNode.getRolesFromSettings(settings) + .stream() + .anyMatch(role -> !DiscoveryNodeRole.SEARCH_ROLE.equals(role))) { throw new SettingsException( "Unable to initialize the " + DiscoveryNodeRole.SEARCH_ROLE.roleName() From 30ddaea8d91c03d884f8f7115036bca6325b1932 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Tue, 19 Mar 2024 18:16:13 +0530 Subject: [PATCH 02/22] Refactor TransferManager interface to RemoteStoreFileTrackerAdapter Signed-off-by: Shreyansh Ray --- .../org/opensearch/index/IndexService.java | 12 ++++++++ .../index/store/CompositeDirectory.java | 28 +++++++++++-------- .../store/CompositeDirectoryFactory.java | 11 +------- ...rectoryRemoteStoreFileTrackerAdapter.java} | 13 ++++++--- ...ava => RemoteStoreFileTrackerAdapter.java} | 2 +- 5 files changed, 39 insertions(+), 27 deletions(-) rename server/src/main/java/org/opensearch/index/store/{CompositeDirectoryTransferManager.java => CompositeDirectoryRemoteStoreFileTrackerAdapter.java} (84%) rename server/src/main/java/org/opensearch/index/store/{TransferManager.java => RemoteStoreFileTrackerAdapter.java} (94%) diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index e501d7eff3f81..3baec81d1e4c5 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -41,6 +41,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.Accountable; import org.opensearch.client.Client; +import org.opensearch.cluster.metadata.ComposableIndexTemplate; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.node.DiscoveryNode; @@ -92,6 +93,7 @@ import org.opensearch.index.shard.ShardPath; import org.opensearch.index.similarity.SimilarityService; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; +import org.opensearch.index.store.CompositeDirectory; import org.opensearch.index.store.Store; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogFactory; @@ -109,6 +111,7 @@ import org.opensearch.search.aggregations.support.ValuesSourceRegistry; import org.opensearch.threadpool.ThreadPool; +import java.awt.*; import java.io.Closeable; import java.io.IOException; import java.nio.file.Path; @@ -495,6 +498,7 @@ public synchronized IndexShard createShard( } }; Store remoteStore = null; +<<<<<<< HEAD boolean seedRemote = false; if (targetNode.isRemoteStoreNode()) { final Directory remoteDirectory; @@ -516,6 +520,11 @@ public synchronized IndexShard createShard( this.indexSettings.getRemoteStorePathStrategy() ); } +======= + Directory remoteDirectory = null; + if (this.indexSettings.isRemoteStoreEnabled()) { + remoteDirectory = remoteDirectoryFactory.newDirectory(this.indexSettings, path); +>>>>>>> f1cd4e4895d (Refactor TransferManager interface to RemoteStoreFileTrackerAdapter) remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path); } else { // Disallow shards with remote store based settings to be created on non-remote store enabled nodes @@ -531,6 +540,9 @@ public synchronized IndexShard createShard( } Directory directory = directoryFactory.newDirectory(this.indexSettings, path); + if (directory instanceof CompositeDirectory) { + ((CompositeDirectory) directory).setRemoteDirectory(remoteDirectory); + } store = new Store( shardId, this.indexSettings, diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 080b42a2b65a2..ba134ad9344f3 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -8,13 +8,13 @@ package org.opensearch.index.store; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; -import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.utils.filetracker.FileState; @@ -27,14 +27,18 @@ public class CompositeDirectory extends FilterDirectory { private final FSDirectory localDirectory; - private final TransferManager transferManager; + private final RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter; private final FileCache fileCache; - public CompositeDirectory(FSDirectory localDirectory, BlobContainer blobContainer, FileCache fileCache) { + public CompositeDirectory(FSDirectory localDirectory, FileCache fileCache) { super(localDirectory); this.localDirectory = localDirectory; this.fileCache = fileCache; - this.transferManager = new CompositeDirectoryTransferManager(fileCache, blobContainer); + this.remoteStoreFileTrackerAdapter = new CompositeDirectoryRemoteStoreFileTrackerAdapter(fileCache); + } + + public void setRemoteDirectory(Directory remoteDirectory) { + ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).setRemoteDirectory(remoteDirectory); } @Override @@ -45,7 +49,7 @@ public String[] listAll() throws IOException { @Override public void deleteFile(String name) throws IOException { super.deleteFile(name); - transferManager.removeFileFromTracker(name); + remoteStoreFileTrackerAdapter.removeFileFromTracker(name); fileCache.remove(localDirectory.getDirectory().resolve(name)); } @@ -56,7 +60,7 @@ public long fileLength(String name) throws IOException { @Override public IndexOutput createOutput(String name, IOContext context) throws IOException { - transferManager.trackFile(name, FileState.DISK, FileType.NON_BLOCK); + remoteStoreFileTrackerAdapter.trackFile(name, FileState.DISK, FileType.NON_BLOCK); return localDirectory.createOutput(name, context); } @@ -78,17 +82,17 @@ public void syncMetaData() throws IOException { @Override public void rename(String source, String dest) throws IOException { localDirectory.rename(source, dest); - transferManager.trackFile(dest, transferManager.getFileState(source), transferManager.getFileType(source)); - transferManager.removeFileFromTracker(source); + remoteStoreFileTrackerAdapter.trackFile(dest, remoteStoreFileTrackerAdapter.getFileState(source), remoteStoreFileTrackerAdapter.getFileType(source)); + remoteStoreFileTrackerAdapter.removeFileFromTracker(source); } @Override public IndexInput openInput(String name, IOContext context) throws IOException { - if (!transferManager.isFilePresent(name)) { + if (!remoteStoreFileTrackerAdapter.isFilePresent(name)) { return localDirectory.openInput(name, context); } IndexInput indexInput = null; - switch (transferManager.getFileState(name)) { + switch (remoteStoreFileTrackerAdapter.getFileState(name)) { case DISK: indexInput = localDirectory.openInput(name, context); break; @@ -121,8 +125,8 @@ public Set getPendingDeletions() throws IOException { public void afterSyncToRemote(Collection files) throws IOException { for (String fileName : files) { - if (transferManager.isFilePresent(fileName) && !transferManager.getFileState(fileName).equals(FileState.CACHE)) { - transferManager.updateFileState(fileName, FileState.CACHE); + if (remoteStoreFileTrackerAdapter.isFilePresent(fileName) && !remoteStoreFileTrackerAdapter.getFileState(fileName).equals(FileState.CACHE)) { + remoteStoreFileTrackerAdapter.updateFileState(fileName, FileState.CACHE); } fileCache.put(localDirectory.getDirectory().resolve(fileName), new CachedIndexInput() { @Override diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java index 52095614f56a9..bed6e2454574a 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java @@ -36,17 +36,8 @@ public CompositeDirectoryFactory(Supplier repositoriesServi @Override public Directory newDirectory(IndexSettings indexSettings, ShardPath shardPath) throws IOException { - String repositoryName = indexSettings.getRemoteStoreRepository(); - Repository repository = repositoriesService.get().repository(repositoryName); - BlobStoreRepository blobStoreRepository = (BlobStoreRepository) repository; - String shardId = String.valueOf(shardPath.getShardId().getId()); - String indexUUID = indexSettings.getIndex().getUUID(); - BlobPath blobPath = blobStoreRepository.basePath().add(indexUUID).add(shardId).add("segments").add("data"); - final BlobContainer blobContainer = blobStoreRepository.blobStore().blobContainer(blobPath); - final Path location = shardPath.resolveIndex(); final FSDirectory primaryDirectory = FSDirectory.open(location); - - return new CompositeDirectory(primaryDirectory, blobContainer, remoteStoreFileCache); + return new CompositeDirectory(primaryDirectory, remoteStoreFileCache); } } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryTransferManager.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java similarity index 84% rename from server/src/main/java/org/opensearch/index/store/CompositeDirectoryTransferManager.java rename to server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java index 4e10a6db80db3..b6ae712579375 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryTransferManager.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java @@ -8,6 +8,7 @@ package org.opensearch.index.store; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.index.store.remote.filecache.FileCache; @@ -19,18 +20,22 @@ import java.util.HashMap; import java.util.Map; -public class CompositeDirectoryTransferManager implements TransferManager { +public class CompositeDirectoryRemoteStoreFileTrackerAdapter implements RemoteStoreFileTrackerAdapter { private FileCache fileCache; private Map fileTracker; - private BlobContainer blobContainer; + private RemoteSegmentStoreDirectory remoteDirectory; - public CompositeDirectoryTransferManager(FileCache fileCache, BlobContainer blobContainer) { + public CompositeDirectoryRemoteStoreFileTrackerAdapter(FileCache fileCache) { this.fileCache = fileCache; - this.blobContainer = blobContainer; + remoteDirectory = null; this.fileTracker = new HashMap<>(); } + public void setRemoteDirectory(Directory remoteDirectory) { + this.remoteDirectory = (RemoteSegmentStoreDirectory) remoteDirectory; + } + @Override public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) { // TODO - This function will fetch the requested data from blobContainer diff --git a/server/src/main/java/org/opensearch/index/store/TransferManager.java b/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java similarity index 94% rename from server/src/main/java/org/opensearch/index/store/TransferManager.java rename to server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java index 5029c51a34e34..9f3ef8a5571d3 100644 --- a/server/src/main/java/org/opensearch/index/store/TransferManager.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java @@ -13,7 +13,7 @@ import org.opensearch.index.store.remote.utils.filetracker.FileState; import org.opensearch.index.store.remote.utils.filetracker.FileType; -public interface TransferManager { +public interface RemoteStoreFileTrackerAdapter { IndexInput fetchBlob(BlobFetchRequest blobFetchRequest); void trackFile(String name, FileState fileState, FileType fileType); From a7361d23c5afae1eabbb1e4ede783834f73e64a9 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Wed, 20 Mar 2024 12:16:30 +0530 Subject: [PATCH 03/22] Implement block level fetch for Composite Directory Signed-off-by: Shreyansh Ray --- .../index/store/CompositeDirectory.java | 7 +- .../store/CompositeDirectoryFactory.java | 9 +- ...irectoryRemoteStoreFileTrackerAdapter.java | 19 +++- .../store/RemoteSegmentStoreDirectory.java | 7 +- .../store/RemoteStoreFileTrackerAdapter.java | 4 +- .../OnDemandCompositeBlockIndexInput.java | 93 +++++++++++++++++++ 6 files changed, 129 insertions(+), 10 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index ba134ad9344f3..66128299a186b 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -15,6 +15,7 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; +import org.opensearch.index.store.remote.file.OnDemandCompositeBlockIndexInput; import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.utils.filetracker.FileState; @@ -29,12 +30,14 @@ public class CompositeDirectory extends FilterDirectory { private final FSDirectory localDirectory; private final RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter; private final FileCache fileCache; + private final FSDirectory localCacheDirectory; - public CompositeDirectory(FSDirectory localDirectory, FileCache fileCache) { + public CompositeDirectory(FSDirectory localDirectory, FSDirectory localCacheDirectory, FileCache fileCache) { super(localDirectory); this.localDirectory = localDirectory; this.fileCache = fileCache; this.remoteStoreFileTrackerAdapter = new CompositeDirectoryRemoteStoreFileTrackerAdapter(fileCache); + this.localCacheDirectory = localCacheDirectory; } public void setRemoteDirectory(Directory remoteDirectory) { @@ -102,7 +105,7 @@ public IndexInput openInput(String name, IOContext context) throws IOException { break; case REMOTE_ONLY: - // TODO - return an implementation of OnDemandBlockIndexInput where the fetchBlock method is implemented + indexInput = new OnDemandCompositeBlockIndexInput(remoteStoreFileTrackerAdapter, name, localCacheDirectory); break; } return indexInput; diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java index bed6e2454574a..9f4fa5b959ed8 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java @@ -21,10 +21,12 @@ import org.opensearch.repositories.blobstore.BlobStoreRepository; import java.io.IOException; +import java.nio.file.Files; import java.nio.file.Path; import java.util.function.Supplier; public class CompositeDirectoryFactory implements IndexStorePlugin.DirectoryFactory { + private static String CACHE_LOCATION = "remote_cache"; private final Supplier repositoriesService; private final FileCache remoteStoreFileCache; @@ -36,8 +38,9 @@ public CompositeDirectoryFactory(Supplier repositoriesServi @Override public Directory newDirectory(IndexSettings indexSettings, ShardPath shardPath) throws IOException { - final Path location = shardPath.resolveIndex(); - final FSDirectory primaryDirectory = FSDirectory.open(location); - return new CompositeDirectory(primaryDirectory, remoteStoreFileCache); + final FSDirectory primaryDirectory = FSDirectory.open(shardPath.resolveIndex()); + final FSDirectory localCacheDirectory = FSDirectory.open(Files.createDirectories(shardPath.getDataPath().resolve(CACHE_LOCATION))); + localCacheDirectory.syncMetaData(); + return new CompositeDirectory(primaryDirectory, localCacheDirectory, remoteStoreFileCache); } } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java index b6ae712579375..1b205aed88dcb 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java @@ -13,10 +13,12 @@ import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.utils.BlobFetchRequest; +import org.opensearch.index.store.remote.utils.TransferManager; import org.opensearch.index.store.remote.utils.filetracker.FileState; import org.opensearch.index.store.remote.utils.filetracker.FileTrackingInfo; import org.opensearch.index.store.remote.utils.filetracker.FileType; +import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -36,10 +38,21 @@ public void setRemoteDirectory(Directory remoteDirectory) { this.remoteDirectory = (RemoteSegmentStoreDirectory) remoteDirectory; } + public String getUploadedFileName(String name) { + return remoteDirectory.getExistingRemoteFilename(name); + } + + public long getFileLength(String name) { + try { + return remoteDirectory.fileLength(name); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + @Override - public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) { - // TODO - This function will fetch the requested data from blobContainer - return null; + public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOException { + return new TransferManager(remoteDirectory.getDataDirectoryBlobContainer(), fileCache).fetchBlob(blobFetchRequest); } public void trackFile(String name, FileState fileState, FileType fileType) { 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 8c0ecb4cc783a..868d1803de152 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -25,6 +25,7 @@ import org.apache.lucene.util.Version; import org.opensearch.common.UUIDs; import org.opensearch.common.annotation.PublicApi; +import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.logging.Loggers; @@ -136,6 +137,10 @@ public RemoteSegmentStoreDirectory( init(); } + public BlobContainer getDataDirectoryBlobContainer() { + return remoteDataDirectory.getBlobContainer(); + } + /** * 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 @@ -698,7 +703,7 @@ private String getChecksumOfLocalFile(Directory directory, String file) throws I } } - private String getExistingRemoteFilename(String localFilename) { + public String getExistingRemoteFilename(String localFilename) { if (segmentsUploadedToRemoteStore.containsKey(localFilename)) { return segmentsUploadedToRemoteStore.get(localFilename).uploadedFilename; } else { diff --git a/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java b/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java index 9f3ef8a5571d3..29f05c8dc60e8 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java @@ -13,8 +13,10 @@ import org.opensearch.index.store.remote.utils.filetracker.FileState; import org.opensearch.index.store.remote.utils.filetracker.FileType; +import java.io.IOException; + public interface RemoteStoreFileTrackerAdapter { - IndexInput fetchBlob(BlobFetchRequest blobFetchRequest); + IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOException; void trackFile(String name, FileState fileState, FileType fileType); diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java new file mode 100644 index 0000000000000..8b8975be7a46b --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java @@ -0,0 +1,93 @@ +/* + * 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.store.remote.file; + +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.IndexInput; +import org.opensearch.index.store.CompositeDirectoryRemoteStoreFileTrackerAdapter; +import org.opensearch.index.store.RemoteStoreFileTrackerAdapter; +import org.opensearch.index.store.remote.utils.BlobFetchRequest; + +import java.io.IOException; + +public class OnDemandCompositeBlockIndexInput extends OnDemandBlockIndexInput { + + private final RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter; + private final String fileName; + private final Long originalFileSize; + private final FSDirectory directory; + + public OnDemandCompositeBlockIndexInput(RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName, FSDirectory directory) { + this( + OnDemandBlockIndexInput.builder(). + resourceDescription("OnDemandCompositeBlockIndexInput"). + isClone(false). + offset(0L). + length(getFileLength(remoteStoreFileTrackerAdapter, fileName)), + remoteStoreFileTrackerAdapter, + fileName, + directory); + } + + public OnDemandCompositeBlockIndexInput(Builder builder, RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName, FSDirectory directory) { + super(builder); + this.remoteStoreFileTrackerAdapter = remoteStoreFileTrackerAdapter; + this.directory = null; + this.fileName = fileName; + originalFileSize = getFileLength(remoteStoreFileTrackerAdapter, fileName); + } + + @Override + protected OnDemandCompositeBlockIndexInput buildSlice(String sliceDescription, long offset, long length) { + return new OnDemandCompositeBlockIndexInput( + OnDemandBlockIndexInput.builder(). + blockSizeShift(blockSizeShift). + isClone(true). + offset(this.offset + offset). + length(length). + resourceDescription(sliceDescription), + remoteStoreFileTrackerAdapter, + fileName, + directory + ); + } + + @Override + protected IndexInput fetchBlock(int blockId) throws IOException { + final String uploadedFileName = ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getUploadedFileName(fileName); + final String blockFileName = uploadedFileName + "." + blockId; + final long blockStart = getBlockStart(blockId); + final long length = getActualBlockSize(blockId); + + BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() + .position(blockStart) + .length(length) + .blobName(uploadedFileName) + .directory(directory) + .fileName(blockFileName) + .build(); + return remoteStoreFileTrackerAdapter.fetchBlob(blobFetchRequest); + } + + @Override + public OnDemandBlockIndexInput clone() { + OnDemandCompositeBlockIndexInput clone = buildSlice("clone", 0L, this.length); + // ensures that clones may be positioned at the same point as the blocked file they were cloned from + clone.cloneBlock(this); + return clone; + } + + private long getActualBlockSize(int blockId) { + return (blockId != getBlock(originalFileSize - 1)) ? blockSize : getBlockOffset(originalFileSize - 1) + 1; + } + + private static long getFileLength(RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName) { + return ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getFileLength(fileName); + } +} From 54beed2469ae331ad811cb9fda3a4b06833c6d4f Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Mon, 1 Apr 2024 12:45:57 +0530 Subject: [PATCH 04/22] Removed CACHE state from FileTracker Signed-off-by: Shreyansh Ray --- .../shard/RemoteStoreRefreshListener.java | 3 +- .../index/store/CompositeDirectory.java | 238 +++++++++++++----- .../store/CompositeDirectoryFactory.java | 5 +- ...irectoryRemoteStoreFileTrackerAdapter.java | 19 +- .../OnDemandCompositeBlockIndexInput.java | 14 +- .../remote/utils/filetracker/FileState.java | 9 +- 6 files changed, 220 insertions(+), 68 deletions(-) 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 f3c2d27c20b58..255835e44b964 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -297,11 +297,12 @@ public void onFailure(Exception e) { } }, latch); + Collection segmentsToRefresh = localSegmentsPostRefresh.stream().filter(file -> !skipUpload(file)).collect(Collectors.toList()); // Start the segments files upload uploadNewSegments(localSegmentsPostRefresh, localSegmentsSizeMap, segmentUploadsCompletedListener); Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); if (directory instanceof CompositeDirectory) { - ((CompositeDirectory) directory).afterSyncToRemote(localSegmentsPostRefresh); + ((CompositeDirectory) directory).afterSyncToRemote(segmentsToRefresh); } if (latch.await( remoteStoreSettings.getClusterRemoteSegmentTransferTimeout().millis(), diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 66128299a186b..3df88e1ac2425 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -8,6 +8,8 @@ package org.opensearch.index.store; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FilterDirectory; @@ -16,103 +18,208 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; import org.opensearch.index.store.remote.file.OnDemandCompositeBlockIndexInput; -import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.utils.filetracker.FileState; import org.opensearch.index.store.remote.utils.filetracker.FileType; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; +import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; public class CompositeDirectory extends FilterDirectory { + private static final Logger logger = LogManager.getLogger(CompositeDirectory.class); private final FSDirectory localDirectory; private final RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter; private final FileCache fileCache; - private final FSDirectory localCacheDirectory; + private final AtomicBoolean isRemoteDirectorySet; + private final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); + private final ReentrantReadWriteLock.WriteLock writeLock = readWriteLock.writeLock(); + private final ReentrantReadWriteLock.ReadLock readLock = readWriteLock.readLock(); - public CompositeDirectory(FSDirectory localDirectory, FSDirectory localCacheDirectory, FileCache fileCache) { + public CompositeDirectory(FSDirectory localDirectory, FileCache fileCache) { super(localDirectory); this.localDirectory = localDirectory; this.fileCache = fileCache; this.remoteStoreFileTrackerAdapter = new CompositeDirectoryRemoteStoreFileTrackerAdapter(fileCache); - this.localCacheDirectory = localCacheDirectory; + isRemoteDirectorySet = new AtomicBoolean(false); } public void setRemoteDirectory(Directory remoteDirectory) { - ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).setRemoteDirectory(remoteDirectory); + logger.trace("Setting remote Directory ..."); + if (!isRemoteDirectorySet()){ + ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).setRemoteDirectory(remoteDirectory); + isRemoteDirectorySet.set(true); + } } @Override public String[] listAll() throws IOException { - return localDirectory.listAll(); + logger.trace("listAll() called ..."); + readLock.lock(); + try { + String[] remoteFiles = new String[0]; + String[] localFiles = localDirectory.listAll(); + if (isRemoteDirectorySet()) + remoteFiles = ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getRemoteFiles(); + logger.trace("LocalDirectory files : " + Arrays.toString(localFiles)); + logger.trace("Remote Directory files : " + Arrays.toString(remoteFiles)); + Set allFiles = new HashSet<>(Arrays.asList(localFiles)); + allFiles.addAll(Arrays.asList(remoteFiles)); + + Set localLuceneFiles = allFiles.stream() + .filter(file -> !isBlockFile(file)) + .collect(Collectors.toUnmodifiableSet()); + String[] files = new String[localLuceneFiles.size()]; + localLuceneFiles.toArray(files); + Arrays.sort(files); + + logger.trace("listAll() returns : " + Arrays.toString(files)); + + return files; + } finally { + readLock.unlock(); + } } @Override public void deleteFile(String name) throws IOException { - super.deleteFile(name); - remoteStoreFileTrackerAdapter.removeFileFromTracker(name); - fileCache.remove(localDirectory.getDirectory().resolve(name)); + logger.trace("deleteFile() called " + name); + writeLock.lock(); + try { + localDirectory.deleteFile(name); + remoteStoreFileTrackerAdapter.removeFileFromTracker(name); + fileCache.remove(localDirectory.getDirectory().resolve(name)); + logFileTracker(); + } finally { + writeLock.unlock(); + } } @Override public long fileLength(String name) throws IOException { - return localDirectory.fileLength(name); + logger.trace("fileLength() called " + name); + readLock.lock(); + try { + if(remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) + { + logger.trace("fileLength from Local " + localDirectory.fileLength(name)); + return localDirectory.fileLength(name); + } + else + { + logger.trace("fileLength from Remote " + ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getFileLength(name)); + return ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getFileLength(name); + } + } finally { + readLock.unlock(); + } } @Override public IndexOutput createOutput(String name, IOContext context) throws IOException { - remoteStoreFileTrackerAdapter.trackFile(name, FileState.DISK, FileType.NON_BLOCK); - return localDirectory.createOutput(name, context); + logger.trace("createOutput() called " + name); + writeLock.lock(); + try { + remoteStoreFileTrackerAdapter.trackFile(name, FileState.DISK, FileType.NON_BLOCK); + logFileTracker(); + return localDirectory.createOutput(name, context); + } finally { + writeLock.unlock(); + } } @Override public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException { - return localDirectory.createTempOutput(prefix, suffix, context); + logger.trace("createOutput() called " + prefix + "," + suffix); + writeLock.lock(); + try { + return localDirectory.createTempOutput(prefix, suffix, context); + } finally { + writeLock.unlock(); + } } @Override public void sync(Collection names) throws IOException { - localDirectory.sync(names); + logger.trace("sync() called " + names); + writeLock.lock(); + try { + Collection newLocalFiles = new ArrayList<>(); + for (String name : names) { + if(remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) + newLocalFiles.add(name); + } + logger.trace("Synced files : " + newLocalFiles); + localDirectory.sync(newLocalFiles); + } finally { + writeLock.unlock(); + } } @Override public void syncMetaData() throws IOException { - localDirectory.syncMetaData(); + logger.trace("syncMetaData() called "); + writeLock.lock(); + try { + localDirectory.syncMetaData(); + } finally { + writeLock.unlock(); + } } @Override public void rename(String source, String dest) throws IOException { - localDirectory.rename(source, dest); - remoteStoreFileTrackerAdapter.trackFile(dest, remoteStoreFileTrackerAdapter.getFileState(source), remoteStoreFileTrackerAdapter.getFileType(source)); - remoteStoreFileTrackerAdapter.removeFileFromTracker(source); + logger.trace("rename() called " + source + " -> " + dest); + writeLock.lock(); + try { + localDirectory.rename(source, dest); + remoteStoreFileTrackerAdapter.trackFile(dest, remoteStoreFileTrackerAdapter.getFileState(source), remoteStoreFileTrackerAdapter.getFileType(source)); + remoteStoreFileTrackerAdapter.removeFileFromTracker(source); + logFileTracker(); + } finally { + writeLock.unlock(); + } } @Override public IndexInput openInput(String name, IOContext context) throws IOException { - if (!remoteStoreFileTrackerAdapter.isFilePresent(name)) { - return localDirectory.openInput(name, context); - } - IndexInput indexInput = null; - switch (remoteStoreFileTrackerAdapter.getFileState(name)) { - case DISK: - indexInput = localDirectory.openInput(name, context); - break; - - case CACHE: - indexInput = fileCache.get(localDirectory.getDirectory().resolve(name)).getIndexInput(); - break; - - case REMOTE_ONLY: - indexInput = new OnDemandCompositeBlockIndexInput(remoteStoreFileTrackerAdapter, name, localCacheDirectory); - break; + logger.trace("openInput() called " + name); + writeLock.lock(); + try { + if (!remoteStoreFileTrackerAdapter.isFilePresent(name)) { + //Print filename to check which file is not present in tracker + logger.trace("File not found in tracker"); + return localDirectory.openInput(name, context); + } + IndexInput indexInput = null; + switch (remoteStoreFileTrackerAdapter.getFileState(name)) { + case DISK: + logger.trace("File found in disk "); + indexInput = localDirectory.openInput(name, context); + break; + + case REMOTE: + logger.trace("File to be fetched from Remote "); + indexInput = new OnDemandCompositeBlockIndexInput(remoteStoreFileTrackerAdapter, name, localDirectory); + break; + } + return indexInput; + } finally { + writeLock.unlock(); } - return indexInput; } @Override public Lock obtainLock(String name) throws IOException { + logger.trace("obtainLock() called " + name); return localDirectory.obtainLock(name); } @@ -127,33 +234,48 @@ public Set getPendingDeletions() throws IOException { } public void afterSyncToRemote(Collection files) throws IOException { + logger.trace("afterSyncToRemote called for " + files); + if(!isRemoteDirectorySet()) + throw new UnsupportedOperationException("Cannot perform afterSyncToRemote if Remote Directory is not set"); + List delFiles = new ArrayList<>(); for (String fileName : files) { - if (remoteStoreFileTrackerAdapter.isFilePresent(fileName) && !remoteStoreFileTrackerAdapter.getFileState(fileName).equals(FileState.CACHE)) { - remoteStoreFileTrackerAdapter.updateFileState(fileName, FileState.CACHE); - } - fileCache.put(localDirectory.getDirectory().resolve(fileName), new CachedIndexInput() { - @Override - public IndexInput getIndexInput() throws IOException { - return localDirectory.openInput(fileName, IOContext.READ); + if (isSegmentsOrLockFile(fileName)) + continue; + writeLock.lock(); + try { + if (remoteStoreFileTrackerAdapter.isFilePresent(fileName) && remoteStoreFileTrackerAdapter.getFileState(fileName).equals(FileState.DISK)) { + remoteStoreFileTrackerAdapter.updateFileState(fileName, FileState.REMOTE); } + } finally { + writeLock.unlock(); + } + localDirectory.deleteFile(fileName); + delFiles.add(fileName); + } + logger.trace("Files removed form local " + delFiles); + logFileTracker(); + } - @Override - public long length() { - try { - return localDirectory.fileLength(fileName); - } catch (IOException e) { - throw new RuntimeException(e); - } - } + private boolean isSegmentsOrLockFile(String fileName) { + if (fileName.startsWith("segments_") || + fileName.endsWith(".si") || + fileName.endsWith(".lock")) + return true; + return false; + } - @Override - public boolean isClosed() { - return false; - } + private boolean isBlockFile(String fileName) { + if (fileName.contains("_block_")) + return true; + return false; + } - @Override - public void close() {} - }); - } + private boolean isRemoteDirectorySet() { + return isRemoteDirectorySet.get(); + } + + public void logFileTracker() { + String res = ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).logFileTracker(); + logger.trace(res); } } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java index 9f4fa5b959ed8..550f89daf34ca 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java @@ -26,7 +26,6 @@ import java.util.function.Supplier; public class CompositeDirectoryFactory implements IndexStorePlugin.DirectoryFactory { - private static String CACHE_LOCATION = "remote_cache"; private final Supplier repositoriesService; private final FileCache remoteStoreFileCache; @@ -39,8 +38,6 @@ public CompositeDirectoryFactory(Supplier repositoriesServi @Override public Directory newDirectory(IndexSettings indexSettings, ShardPath shardPath) throws IOException { final FSDirectory primaryDirectory = FSDirectory.open(shardPath.resolveIndex()); - final FSDirectory localCacheDirectory = FSDirectory.open(Files.createDirectories(shardPath.getDataPath().resolve(CACHE_LOCATION))); - localCacheDirectory.syncMetaData(); - return new CompositeDirectory(primaryDirectory, localCacheDirectory, remoteStoreFileCache); + return new CompositeDirectory(primaryDirectory, remoteStoreFileCache); } } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java index 1b205aed88dcb..d7e2603607642 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java @@ -10,7 +10,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.IndexInput; -import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.utils.BlobFetchRequest; import org.opensearch.index.store.remote.utils.TransferManager; @@ -96,4 +95,22 @@ public FileType getFileType(String name) { public boolean isFilePresent(String name) { return fileTracker.containsKey(name); } + + public String[] getRemoteFiles() throws IOException { + String[] remoteFiles; + try { + remoteFiles = remoteDirectory.listAll(); + } catch (Exception e) { + remoteFiles = new String[0]; + } + return remoteFiles; + } + + public String logFileTracker() { + String result = ""; + for (Map.Entry entry : fileTracker.entrySet()) { + result += entry.getKey() + " : " + entry.getValue().getFileType().name() + " , " + entry.getValue().getFileState().name() +"\n"; + } + return result; + } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java index 8b8975be7a46b..a364d745a2075 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java @@ -8,6 +8,8 @@ package org.opensearch.index.store.remote.file; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.IndexInput; import org.opensearch.index.store.CompositeDirectoryRemoteStoreFileTrackerAdapter; @@ -18,6 +20,7 @@ public class OnDemandCompositeBlockIndexInput extends OnDemandBlockIndexInput { + private static final Logger logger = LogManager.getLogger(OnDemandCompositeBlockIndexInput.class); private final RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter; private final String fileName; private final Long originalFileSize; @@ -38,7 +41,7 @@ public OnDemandCompositeBlockIndexInput(RemoteStoreFileTrackerAdapter remoteStor public OnDemandCompositeBlockIndexInput(Builder builder, RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName, FSDirectory directory) { super(builder); this.remoteStoreFileTrackerAdapter = remoteStoreFileTrackerAdapter; - this.directory = null; + this.directory = directory; this.fileName = fileName; originalFileSize = getFileLength(remoteStoreFileTrackerAdapter, fileName); } @@ -60,10 +63,17 @@ protected OnDemandCompositeBlockIndexInput buildSlice(String sliceDescription, l @Override protected IndexInput fetchBlock(int blockId) throws IOException { + logger.trace("fetchBlock called with blockId -> " + blockId); final String uploadedFileName = ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getUploadedFileName(fileName); - final String blockFileName = uploadedFileName + "." + blockId; + final String blockFileName = fileName + "_block_" + blockId; final long blockStart = getBlockStart(blockId); final long length = getActualBlockSize(blockId); + logger.trace("File: " + uploadedFileName + + ", Block File: " + blockFileName + + ", BlockStart: " + blockStart + + ", Length: " + length + + ", BlockSize: " + blockSize + + ", OriginalFileSize: " + originalFileSize); BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() .position(blockStart) diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java index 564bb87e03857..73b39bac35f35 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java @@ -9,7 +9,12 @@ package org.opensearch.index.store.remote.utils.filetracker; public enum FileState { + /** + * DISK State means that currently the file is present only locally and has not yet been uploaded to the Remote Store + */ DISK, - CACHE, - REMOTE_ONLY; + /** + * REMOTE State means that the file has been successfully uploaded to the Remote Store and is safe to be removed locally + */ + REMOTE; } From 87772f051ddf046b0f96c2950806a95397daf62f Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Mon, 1 Apr 2024 15:00:27 +0530 Subject: [PATCH 05/22] Fixes after latest pull Signed-off-by: Shreyansh Ray --- .../common/blobstore/BlobContainer.java | 3 + .../common/blobstore/BlobMetadata.java | 3 + .../common/blobstore/DeleteResult.java | 3 + .../org/opensearch/index/IndexService.java | 12 +--- .../shard/RemoteStoreRefreshListener.java | 4 +- .../index/store/CompositeDirectory.java | 58 ++++++++------- .../store/CompositeDirectoryFactory.java | 6 -- ...irectoryRemoteStoreFileTrackerAdapter.java | 7 +- .../OnDemandCompositeBlockIndexInput.java | 71 ++++++++++++------- 9 files changed, 93 insertions(+), 74 deletions(-) diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index a2e4199029ef4..b21f6541dbdfd 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -34,6 +34,7 @@ import org.opensearch.common.Nullable; import org.opensearch.common.annotation.ExperimentalApi; +import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.action.ActionListener; import java.io.IOException; @@ -50,6 +51,7 @@ * * @opensearch.internal */ +@PublicApi(since = "2.3.0") public interface BlobContainer { /** @@ -277,6 +279,7 @@ default void writeBlobAtomicWithMetadata( /** * The type representing sort order of blob names */ + @PublicApi(since = "2.3.0") enum BlobNameSortOrder { LEXICOGRAPHIC(Comparator.comparing(BlobMetadata::name)); diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java index 37c70365b6a11..4fdfb413d24d7 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java @@ -32,11 +32,14 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.annotation.PublicApi; + /** * An interface for providing basic metadata about a blob. * * @opensearch.internal */ +@PublicApi(since = "2.3.0") public interface BlobMetadata { /** diff --git a/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java b/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java index 3b424c582ebc6..9ee37515cd005 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java +++ b/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java @@ -32,11 +32,14 @@ package org.opensearch.common.blobstore; +import org.opensearch.common.annotation.PublicApi; + /** * The result of deleting multiple blobs from a {@link BlobStore}. * * @opensearch.internal */ +@PublicApi(since = "2.3.0") public final class DeleteResult { public static final DeleteResult ZERO = new DeleteResult(0, 0); diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 3baec81d1e4c5..61309b4e29982 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -41,7 +41,6 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.util.Accountable; import org.opensearch.client.Client; -import org.opensearch.cluster.metadata.ComposableIndexTemplate; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.cluster.metadata.IndexNameExpressionResolver; import org.opensearch.cluster.node.DiscoveryNode; @@ -92,8 +91,8 @@ import org.opensearch.index.shard.ShardNotInPrimaryModeException; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.similarity.SimilarityService; -import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.CompositeDirectory; +import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogFactory; @@ -111,7 +110,6 @@ import org.opensearch.search.aggregations.support.ValuesSourceRegistry; import org.opensearch.threadpool.ThreadPool; -import java.awt.*; import java.io.Closeable; import java.io.IOException; import java.nio.file.Path; @@ -498,10 +496,9 @@ public synchronized IndexShard createShard( } }; Store remoteStore = null; -<<<<<<< HEAD + Directory remoteDirectory = null; boolean seedRemote = false; if (targetNode.isRemoteStoreNode()) { - final Directory remoteDirectory; if (this.indexSettings.isRemoteStoreEnabled()) { remoteDirectory = remoteDirectoryFactory.newDirectory(this.indexSettings, path); } else { @@ -520,11 +517,6 @@ public synchronized IndexShard createShard( this.indexSettings.getRemoteStorePathStrategy() ); } -======= - Directory remoteDirectory = null; - if (this.indexSettings.isRemoteStoreEnabled()) { - remoteDirectory = remoteDirectoryFactory.newDirectory(this.indexSettings, path); ->>>>>>> f1cd4e4895d (Refactor TransferManager interface to RemoteStoreFileTrackerAdapter) remoteStore = new Store(shardId, this.indexSettings, remoteDirectory, lock, Store.OnClose.EMPTY, path); } else { // Disallow shards with remote store based settings to be created on non-remote store enabled nodes 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 255835e44b964..9101d4e066794 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -297,7 +297,9 @@ public void onFailure(Exception e) { } }, latch); - Collection segmentsToRefresh = localSegmentsPostRefresh.stream().filter(file -> !skipUpload(file)).collect(Collectors.toList()); + Collection segmentsToRefresh = localSegmentsPostRefresh.stream() + .filter(file -> !skipUpload(file)) + .collect(Collectors.toList()); // Start the segments files upload uploadNewSegments(localSegmentsPostRefresh, localSegmentsSizeMap, segmentUploadsCompletedListener); Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 3df88e1ac2425..291b499eeb1b4 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -54,8 +54,8 @@ public CompositeDirectory(FSDirectory localDirectory, FileCache fileCache) { public void setRemoteDirectory(Directory remoteDirectory) { logger.trace("Setting remote Directory ..."); - if (!isRemoteDirectorySet()){ - ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).setRemoteDirectory(remoteDirectory); + if (!isRemoteDirectorySet()) { + ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).setRemoteDirectory(remoteDirectory); isRemoteDirectorySet.set(true); } } @@ -67,16 +67,14 @@ public String[] listAll() throws IOException { try { String[] remoteFiles = new String[0]; String[] localFiles = localDirectory.listAll(); - if (isRemoteDirectorySet()) - remoteFiles = ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getRemoteFiles(); + if (isRemoteDirectorySet()) remoteFiles = ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter) + .getRemoteFiles(); logger.trace("LocalDirectory files : " + Arrays.toString(localFiles)); logger.trace("Remote Directory files : " + Arrays.toString(remoteFiles)); Set allFiles = new HashSet<>(Arrays.asList(localFiles)); allFiles.addAll(Arrays.asList(remoteFiles)); - Set localLuceneFiles = allFiles.stream() - .filter(file -> !isBlockFile(file)) - .collect(Collectors.toUnmodifiableSet()); + Set localLuceneFiles = allFiles.stream().filter(file -> !isBlockFile(file)).collect(Collectors.toUnmodifiableSet()); String[] files = new String[localLuceneFiles.size()]; localLuceneFiles.toArray(files); Arrays.sort(files); @@ -108,15 +106,15 @@ public long fileLength(String name) throws IOException { logger.trace("fileLength() called " + name); readLock.lock(); try { - if(remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) - { + if (remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) { logger.trace("fileLength from Local " + localDirectory.fileLength(name)); return localDirectory.fileLength(name); - } - else - { - logger.trace("fileLength from Remote " + ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getFileLength(name)); - return ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getFileLength(name); + } else { + logger.trace( + "fileLength from Remote " + + ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).getFileLength(name) + ); + return ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).getFileLength(name); } } finally { readLock.unlock(); @@ -154,8 +152,7 @@ public void sync(Collection names) throws IOException { try { Collection newLocalFiles = new ArrayList<>(); for (String name : names) { - if(remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) - newLocalFiles.add(name); + if (remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) newLocalFiles.add(name); } logger.trace("Synced files : " + newLocalFiles); localDirectory.sync(newLocalFiles); @@ -181,7 +178,11 @@ public void rename(String source, String dest) throws IOException { writeLock.lock(); try { localDirectory.rename(source, dest); - remoteStoreFileTrackerAdapter.trackFile(dest, remoteStoreFileTrackerAdapter.getFileState(source), remoteStoreFileTrackerAdapter.getFileType(source)); + remoteStoreFileTrackerAdapter.trackFile( + dest, + remoteStoreFileTrackerAdapter.getFileState(source), + remoteStoreFileTrackerAdapter.getFileType(source) + ); remoteStoreFileTrackerAdapter.removeFileFromTracker(source); logFileTracker(); } finally { @@ -195,7 +196,7 @@ public IndexInput openInput(String name, IOContext context) throws IOException { writeLock.lock(); try { if (!remoteStoreFileTrackerAdapter.isFilePresent(name)) { - //Print filename to check which file is not present in tracker + // Print filename to check which file is not present in tracker logger.trace("File not found in tracker"); return localDirectory.openInput(name, context); } @@ -235,15 +236,16 @@ public Set getPendingDeletions() throws IOException { public void afterSyncToRemote(Collection files) throws IOException { logger.trace("afterSyncToRemote called for " + files); - if(!isRemoteDirectorySet()) - throw new UnsupportedOperationException("Cannot perform afterSyncToRemote if Remote Directory is not set"); + if (!isRemoteDirectorySet()) throw new UnsupportedOperationException( + "Cannot perform afterSyncToRemote if Remote Directory is not set" + ); List delFiles = new ArrayList<>(); for (String fileName : files) { - if (isSegmentsOrLockFile(fileName)) - continue; + if (isSegmentsOrLockFile(fileName)) continue; writeLock.lock(); try { - if (remoteStoreFileTrackerAdapter.isFilePresent(fileName) && remoteStoreFileTrackerAdapter.getFileState(fileName).equals(FileState.DISK)) { + if (remoteStoreFileTrackerAdapter.isFilePresent(fileName) + && remoteStoreFileTrackerAdapter.getFileState(fileName).equals(FileState.DISK)) { remoteStoreFileTrackerAdapter.updateFileState(fileName, FileState.REMOTE); } } finally { @@ -257,16 +259,12 @@ public void afterSyncToRemote(Collection files) throws IOException { } private boolean isSegmentsOrLockFile(String fileName) { - if (fileName.startsWith("segments_") || - fileName.endsWith(".si") || - fileName.endsWith(".lock")) - return true; + if (fileName.startsWith("segments_") || fileName.endsWith(".si") || fileName.endsWith(".lock")) return true; return false; } private boolean isBlockFile(String fileName) { - if (fileName.contains("_block_")) - return true; + if (fileName.contains("_block_")) return true; return false; } @@ -275,7 +273,7 @@ private boolean isRemoteDirectorySet() { } public void logFileTracker() { - String res = ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).logFileTracker(); + String res = ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).logFileTracker(); logger.trace(res); } } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java index 550f89daf34ca..8256e22eadbca 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java @@ -10,19 +10,13 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; -import org.opensearch.common.blobstore.BlobContainer; -import org.opensearch.common.blobstore.BlobPath; import org.opensearch.index.IndexSettings; import org.opensearch.index.shard.ShardPath; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.plugins.IndexStorePlugin; import org.opensearch.repositories.RepositoriesService; -import org.opensearch.repositories.Repository; -import org.opensearch.repositories.blobstore.BlobStoreRepository; import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; import java.util.function.Supplier; public class CompositeDirectoryFactory implements IndexStorePlugin.DirectoryFactory { diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java index d7e2603607642..675c6b8c39871 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java @@ -109,7 +109,12 @@ public String[] getRemoteFiles() throws IOException { public String logFileTracker() { String result = ""; for (Map.Entry entry : fileTracker.entrySet()) { - result += entry.getKey() + " : " + entry.getValue().getFileType().name() + " , " + entry.getValue().getFileState().name() +"\n"; + result += entry.getKey() + + " : " + + entry.getValue().getFileType().name() + + " , " + + entry.getValue().getFileState().name() + + "\n"; } return result; } diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java index a364d745a2075..8792e2294a81b 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java @@ -17,6 +17,8 @@ import org.opensearch.index.store.remote.utils.BlobFetchRequest; import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; public class OnDemandCompositeBlockIndexInput extends OnDemandBlockIndexInput { @@ -26,19 +28,29 @@ public class OnDemandCompositeBlockIndexInput extends OnDemandBlockIndexInput { private final Long originalFileSize; private final FSDirectory directory; - public OnDemandCompositeBlockIndexInput(RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName, FSDirectory directory) { + public OnDemandCompositeBlockIndexInput( + RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, + String fileName, + FSDirectory directory + ) { this( - OnDemandBlockIndexInput.builder(). - resourceDescription("OnDemandCompositeBlockIndexInput"). - isClone(false). - offset(0L). - length(getFileLength(remoteStoreFileTrackerAdapter, fileName)), + OnDemandBlockIndexInput.builder() + .resourceDescription("OnDemandCompositeBlockIndexInput") + .isClone(false) + .offset(0L) + .length(getFileLength(remoteStoreFileTrackerAdapter, fileName)), remoteStoreFileTrackerAdapter, fileName, - directory); + directory + ); } - public OnDemandCompositeBlockIndexInput(Builder builder, RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName, FSDirectory directory) { + public OnDemandCompositeBlockIndexInput( + Builder builder, + RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, + String fileName, + FSDirectory directory + ) { super(builder); this.remoteStoreFileTrackerAdapter = remoteStoreFileTrackerAdapter; this.directory = directory; @@ -49,12 +61,12 @@ public OnDemandCompositeBlockIndexInput(Builder builder, RemoteStoreFileTrackerA @Override protected OnDemandCompositeBlockIndexInput buildSlice(String sliceDescription, long offset, long length) { return new OnDemandCompositeBlockIndexInput( - OnDemandBlockIndexInput.builder(). - blockSizeShift(blockSizeShift). - isClone(true). - offset(this.offset + offset). - length(length). - resourceDescription(sliceDescription), + OnDemandBlockIndexInput.builder() + .blockSizeShift(blockSizeShift) + .isClone(true) + .offset(this.offset + offset) + .length(length) + .resourceDescription(sliceDescription), remoteStoreFileTrackerAdapter, fileName, directory @@ -64,21 +76,28 @@ protected OnDemandCompositeBlockIndexInput buildSlice(String sliceDescription, l @Override protected IndexInput fetchBlock(int blockId) throws IOException { logger.trace("fetchBlock called with blockId -> " + blockId); - final String uploadedFileName = ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getUploadedFileName(fileName); + final String uploadedFileName = ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter) + .getUploadedFileName(fileName); final String blockFileName = fileName + "_block_" + blockId; final long blockStart = getBlockStart(blockId); final long length = getActualBlockSize(blockId); - logger.trace("File: " + uploadedFileName + - ", Block File: " + blockFileName + - ", BlockStart: " + blockStart + - ", Length: " + length + - ", BlockSize: " + blockSize + - ", OriginalFileSize: " + originalFileSize); - + logger.trace( + "File: " + + uploadedFileName + + ", Block File: " + + blockFileName + + ", BlockStart: " + + blockStart + + ", Length: " + + length + + ", BlockSize: " + + blockSize + + ", OriginalFileSize: " + + originalFileSize + ); + BlobFetchRequest.BlobPart blobPart = new BlobFetchRequest.BlobPart(uploadedFileName, blockStart, length); BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() - .position(blockStart) - .length(length) - .blobName(uploadedFileName) + .blobParts(new ArrayList<>(Arrays.asList(blobPart))) .directory(directory) .fileName(blockFileName) .build(); @@ -98,6 +117,6 @@ private long getActualBlockSize(int blockId) { } private static long getFileLength(RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName) { - return ((CompositeDirectoryRemoteStoreFileTrackerAdapter)remoteStoreFileTrackerAdapter).getFileLength(fileName); + return ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).getFileLength(fileName); } } From 8fb76d18b1712eb02387e00a61d4d07ecf962ce5 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Fri, 5 Apr 2024 01:56:41 +0530 Subject: [PATCH 06/22] Add new setting for warm, remove store type setting, FileTracker and RemoteStoreFileTrackerAdapter, CompositeDirectoryFactory and update Composite Directory implementation Signed-off-by: Shreyansh Ray --- .../remotestore/CompositeDirectoryIT.java | 7 +- .../metadata/MetadataCreateIndexService.java | 10 +- .../common/blobstore/BlobContainer.java | 3 - .../common/blobstore/BlobMetadata.java | 3 - .../common/blobstore/DeleteResult.java | 3 - .../common/settings/IndexScopedSettings.java | 1 + .../org/opensearch/index/IndexModule.java | 62 +++- .../org/opensearch/index/IndexService.java | 28 +- .../org/opensearch/index/IndexSettings.java | 12 + .../index/store/CompositeDirectory.java | 284 +++++++++++------- .../store/CompositeDirectoryFactory.java | 37 --- ...irectoryRemoteStoreFileTrackerAdapter.java | 121 -------- .../index/store/RemoteDirectory.java | 16 +- .../store/RemoteSegmentStoreDirectory.java | 7 +- .../store/RemoteStoreFileTrackerAdapter.java | 34 --- .../OnDemandCompositeBlockIndexInput.java | 215 +++++++++---- .../store/remote/utils/BlockIOContext.java | 62 ++++ .../index/store/remote/utils/FileType.java | 38 +++ .../remote/utils/filetracker/FileState.java | 20 -- .../utils/filetracker/FileTrackingInfo.java | 27 -- .../remote/utils/filetracker/FileType.java | 18 -- .../opensearch/indices/IndicesService.java | 12 +- .../main/java/org/opensearch/node/Node.java | 3 +- .../opensearch/index/IndexModuleTests.java | 12 +- .../snapshots/SnapshotResiliencyTests.java | 3 +- 25 files changed, 572 insertions(+), 466 deletions(-) delete mode 100644 server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java delete mode 100644 server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java delete mode 100644 server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java delete mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java delete mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java delete mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java index abb291644e479..35499c2f8cc5e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java @@ -14,6 +14,7 @@ import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexShard; @@ -30,9 +31,11 @@ public class CompositeDirectoryIT extends RemoteStoreBaseIntegTestCase { public void testCompositeDirectory() throws Exception { Settings settings = Settings.builder() + .put(super.featureFlagSettings()) + .put(FeatureFlags.WRITEABLE_REMOTE_INDEX, "true") .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), "compositefs") .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), "partial") .build(); assertAcked(client().admin().indices().prepareCreate("test-idx-1").setSettings(settings).get()); GetIndexResponse getIndexResponse = client().admin() @@ -57,7 +60,7 @@ public void testCompositeDirectory() throws Exception { assertEquals("true", indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); assertEquals(REPOSITORY_NAME, indexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); assertEquals(REPOSITORY_2_NAME, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); - assertEquals("compositefs", indexSettings.get("index.store.type")); + assertEquals("partial", indexSettings.get("index.store.locality")); ensureGreen("test-idx-1"); indexData(10, false, "test-idx-1"); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 7b21e39026c26..8999d3c348c51 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -996,7 +996,7 @@ static Settings aggregateIndexSettings( validateStoreTypeSettings(indexSettings); validateRefreshIntervalSettings(request.settings(), clusterSettings); validateTranslogDurabilitySettings(request.settings(), clusterSettings, settings); - validateCompositeFS(request.settings()); + validateIndexStoreLocality(request.settings()); return indexSettings; } @@ -1691,12 +1691,12 @@ static void validateTranslogDurabilitySettings(Settings requestSettings, Cluster } - public static void validateCompositeFS(Settings indexSettings) { - if (indexSettings.get(IndexModule.INDEX_STORE_TYPE_SETTING.getKey(), "") - .equalsIgnoreCase(IndexModule.Type.COMPOSITEFS.getSettingsKey()) + public static void validateIndexStoreLocality(Settings indexSettings) { + if (indexSettings.get(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.FULL.toString()) + .equalsIgnoreCase(IndexModule.DataLocalityType.PARTIAL.toString()) && !FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING)) { throw new IllegalArgumentException( - "ERROR - Composite FS store type can be enabled only if Feature Flag for Writable Remote Index is true" + "index.store.locality can be set to PARTIAL only if Feature Flag for Writable Remote Index is true" ); } } diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java index b21f6541dbdfd..a2e4199029ef4 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobContainer.java @@ -34,7 +34,6 @@ import org.opensearch.common.Nullable; import org.opensearch.common.annotation.ExperimentalApi; -import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.action.ActionListener; import java.io.IOException; @@ -51,7 +50,6 @@ * * @opensearch.internal */ -@PublicApi(since = "2.3.0") public interface BlobContainer { /** @@ -279,7 +277,6 @@ default void writeBlobAtomicWithMetadata( /** * The type representing sort order of blob names */ - @PublicApi(since = "2.3.0") enum BlobNameSortOrder { LEXICOGRAPHIC(Comparator.comparing(BlobMetadata::name)); diff --git a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java index 4fdfb413d24d7..37c70365b6a11 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java +++ b/server/src/main/java/org/opensearch/common/blobstore/BlobMetadata.java @@ -32,14 +32,11 @@ package org.opensearch.common.blobstore; -import org.opensearch.common.annotation.PublicApi; - /** * An interface for providing basic metadata about a blob. * * @opensearch.internal */ -@PublicApi(since = "2.3.0") public interface BlobMetadata { /** diff --git a/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java b/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java index 9ee37515cd005..3b424c582ebc6 100644 --- a/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java +++ b/server/src/main/java/org/opensearch/common/blobstore/DeleteResult.java @@ -32,14 +32,11 @@ package org.opensearch.common.blobstore; -import org.opensearch.common.annotation.PublicApi; - /** * The result of deleting multiple blobs from a {@link BlobStore}. * * @opensearch.internal */ -@PublicApi(since = "2.3.0") public final class DeleteResult { public static final DeleteResult ZERO = new DeleteResult(0, 0); diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 6fe8dec9c21b1..19b3b13dfd396 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -188,6 +188,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { MapperService.INDEX_MAPPING_FIELD_NAME_LENGTH_LIMIT_SETTING, BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING, IndexModule.INDEX_STORE_TYPE_SETTING, + IndexModule.INDEX_STORE_LOCALITY_SETTING, IndexModule.INDEX_STORE_PRE_LOAD_SETTING, IndexModule.INDEX_STORE_HYBRID_MMAP_EXTENSIONS, IndexModule.INDEX_STORE_HYBRID_NIO_EXTENSIONS, diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index 16dce2645b2d3..b4fd7666ba45a 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -74,7 +74,6 @@ import org.opensearch.index.shard.IndexingOperationListener; import org.opensearch.index.shard.SearchOperationListener; import org.opensearch.index.similarity.SimilarityService; -import org.opensearch.index.store.CompositeDirectoryFactory; import org.opensearch.index.store.FsDirectoryFactory; import org.opensearch.index.store.remote.directory.RemoteSnapshotDirectoryFactory; import org.opensearch.index.store.remote.filecache.FileCache; @@ -108,6 +107,8 @@ import java.util.function.Function; import java.util.function.Supplier; +import static org.apache.logging.log4j.util.Strings.toRootUpperCase; + /** * IndexModule represents the central extension point for index level custom implementations like: *
    @@ -142,6 +143,17 @@ public final class IndexModule { Property.NodeScope ); + /** + * Index setting which used to determine how the data is cached locally fully or partially + */ + public static final Setting INDEX_STORE_LOCALITY_SETTING = new Setting<>( + "index.store.data_locality", + DataLocalityType.FULL.name(), + DataLocalityType::getValueOf, + Property.IndexScope, + Property.NodeScope + ); + public static final Setting INDEX_RECOVERY_TYPE_SETTING = new Setting<>( "index.recovery.type", "", @@ -298,6 +310,7 @@ public Iterator> settings() { private final AtomicBoolean frozen = new AtomicBoolean(false); private final BooleanSupplier allowExpensiveQueries; private final Map recoveryStateFactories; + private final FileCache fileCache; /** * Construct the index module for the index with the specified index settings. The index module contains extension points for plugins @@ -316,7 +329,8 @@ public IndexModule( final Map directoryFactories, final BooleanSupplier allowExpensiveQueries, final IndexNameExpressionResolver expressionResolver, - final Map recoveryStateFactories + final Map recoveryStateFactories, + final FileCache fileCache ) { this.indexSettings = indexSettings; this.analysisRegistry = analysisRegistry; @@ -328,6 +342,7 @@ public IndexModule( this.allowExpensiveQueries = allowExpensiveQueries; this.expressionResolver = expressionResolver; this.recoveryStateFactories = recoveryStateFactories; + this.fileCache = fileCache; } /** @@ -507,8 +522,7 @@ public enum Type { MMAPFS("mmapfs"), SIMPLEFS("simplefs"), FS("fs"), - REMOTE_SNAPSHOT("remote_snapshot"), - COMPOSITEFS("compositefs"); + REMOTE_SNAPSHOT("remote_snapshot"); private final String settingsKey; private final boolean deprecated; @@ -579,6 +593,40 @@ public boolean match(Settings settings) { } } + /** + * Indicates the locality of the data - whether it will be cached fully or partially + */ + public enum DataLocalityType { + /** + * Indicates that all the data will be cached locally + */ + FULL, + /** + * Indicates that only a subset of the data will be cached locally + */ + PARTIAL; + + private static final Map LOCALITY_TYPES; + + static { + final Map localityTypes = new HashMap<>(values().length); + for (final DataLocalityType dataLocalityType : values()) { + localityTypes.put(dataLocalityType.name(), dataLocalityType); + } + LOCALITY_TYPES = Collections.unmodifiableMap(localityTypes); + } + + public static DataLocalityType getValueOf(final String localityType) { + Objects.requireNonNull(localityType, "No locality type given."); + final String localityTypeName = toRootUpperCase(localityType.trim()); + final DataLocalityType type = LOCALITY_TYPES.get(localityTypeName); + if (type != null) { + return type; + } + throw new IllegalArgumentException("Unknown Locality Type constant [" + localityType + "]."); + } + } + public static Type defaultStoreType(final boolean allowMmap) { if (allowMmap && Constants.JRE_IS_64BIT && MMapDirectory.UNMAP_SUPPORTED) { return Type.HYBRIDFS; @@ -667,7 +715,8 @@ public IndexService newIndexService( translogFactorySupplier, clusterDefaultRefreshIntervalSupplier, recoverySettings, - remoteStoreSettings + remoteStoreSettings, + fileCache ); success = true; return indexService; @@ -790,9 +839,6 @@ public static Map createBuiltInDirect new RemoteSnapshotDirectoryFactory(repositoriesService, threadPool, remoteStoreFileCache) ); break; - case COMPOSITEFS: - factories.put(type.getSettingsKey(), new CompositeDirectoryFactory(repositoriesService, remoteStoreFileCache)); - break; default: throw new IllegalStateException("No directory factory mapping for built-in type " + type); } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 61309b4e29982..08a9779a23c02 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -39,6 +39,7 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.Accountable; import org.opensearch.client.Client; import org.opensearch.cluster.metadata.IndexMetadata; @@ -55,6 +56,7 @@ import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; import org.opensearch.common.util.BigArrays; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.util.concurrent.AbstractAsyncTask; import org.opensearch.common.util.concurrent.AbstractRunnable; import org.opensearch.common.util.io.IOUtils; @@ -92,8 +94,10 @@ import org.opensearch.index.shard.ShardPath; import org.opensearch.index.similarity.SimilarityService; import org.opensearch.index.store.CompositeDirectory; +import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; +import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.translog.Translog; import org.opensearch.index.translog.TranslogFactory; import org.opensearch.indices.RemoteStoreSettings; @@ -189,6 +193,7 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust private final Supplier clusterDefaultRefreshIntervalSupplier; private final RecoverySettings recoverySettings; private final RemoteStoreSettings remoteStoreSettings; + private final FileCache fileCache; public IndexService( IndexSettings indexSettings, @@ -224,7 +229,8 @@ public IndexService( BiFunction translogFactorySupplier, Supplier clusterDefaultRefreshIntervalSupplier, RecoverySettings recoverySettings, - RemoteStoreSettings remoteStoreSettings + RemoteStoreSettings remoteStoreSettings, + FileCache fileCache ) { super(indexSettings); this.allowExpensiveQueries = allowExpensiveQueries; @@ -302,6 +308,7 @@ public IndexService( this.translogFactorySupplier = translogFactorySupplier; this.recoverySettings = recoverySettings; this.remoteStoreSettings = remoteStoreSettings; + this.fileCache = fileCache; updateFsyncTaskIfNecessary(); } @@ -531,10 +538,23 @@ public synchronized IndexShard createShard( } } - Directory directory = directoryFactory.newDirectory(this.indexSettings, path); - if (directory instanceof CompositeDirectory) { - ((CompositeDirectory) directory).setRemoteDirectory(remoteDirectory); + Directory directory = null; + if (FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING) && + // TODO : Need to remove this check after support for hot indices is added in Composite Directory + this.indexSettings.isStoreLocalityPartial()) { + /** + * Currently Composite Directory only supports local directory to be of type FSDirectory + * The reason is that FileCache currently has it key type as Path + * Composite Directory currently uses FSDirectory's getDirectory() method to fetch and use the Path for operating on FileCache + * TODO : Refactor FileCache to have key in form of String instead of Path. Once that is done we can remove this assertion + */ + Directory localDirectory = directoryFactory.newDirectory(this.indexSettings, path); + assert localDirectory instanceof FSDirectory : "For Composite Directory, local directory must be of type FSDirectory"; + directory = new CompositeDirectory((FSDirectory) localDirectory, (RemoteSegmentStoreDirectory) remoteDirectory, fileCache); + } else { + directory = directoryFactory.newDirectory(this.indexSettings, path); } + store = new Store( shardId, this.indexSettings, diff --git a/server/src/main/java/org/opensearch/index/IndexSettings.java b/server/src/main/java/org/opensearch/index/IndexSettings.java index 613e93698d683..96458ecc49ddc 100644 --- a/server/src/main/java/org/opensearch/index/IndexSettings.java +++ b/server/src/main/java/org/opensearch/index/IndexSettings.java @@ -733,6 +733,7 @@ public static IndexMergePolicy fromString(String text) { private final int numberOfShards; private final ReplicationType replicationType; private final boolean isRemoteStoreEnabled; + private final boolean isStoreLocalityPartial; private volatile TimeValue remoteTranslogUploadBufferInterval; private final String remoteStoreTranslogRepository; private final String remoteStoreRepository; @@ -934,6 +935,10 @@ public IndexSettings(final IndexMetadata indexMetadata, final Settings nodeSetti numberOfShards = settings.getAsInt(IndexMetadata.SETTING_NUMBER_OF_SHARDS, null); replicationType = IndexMetadata.INDEX_REPLICATION_TYPE_SETTING.get(settings); isRemoteStoreEnabled = settings.getAsBoolean(IndexMetadata.SETTING_REMOTE_STORE_ENABLED, false); + isStoreLocalityPartial = settings.get( + IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), + IndexModule.DataLocalityType.FULL.toString() + ).equalsIgnoreCase(IndexModule.DataLocalityType.PARTIAL.toString()); remoteStoreTranslogRepository = settings.get(IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY); remoteTranslogUploadBufferInterval = INDEX_REMOTE_TRANSLOG_BUFFER_INTERVAL_SETTING.get(settings); remoteStoreRepository = settings.get(IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY); @@ -1294,6 +1299,13 @@ public String getRemoteStoreTranslogRepository() { return remoteStoreTranslogRepository; } + /** + * Returns true if the store locality is partial + */ + public boolean isStoreLocalityPartial() { + return isStoreLocalityPartial; + } + /** * Returns true if this is remote/searchable snapshot */ diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 291b499eeb1b4..e668ac31e57ee 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -10,133 +10,165 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; +import org.apache.lucene.store.LockObtainFailedException; import org.opensearch.index.store.remote.file.OnDemandCompositeBlockIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.utils.filetracker.FileState; -import org.opensearch.index.store.remote.utils.filetracker.FileType; +import org.opensearch.index.store.remote.utils.FileType; +import java.io.FileNotFoundException; import java.io.IOException; +import java.nio.file.NoSuchFileException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; -import java.util.List; import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; +/** + * Composite Directory will contain both local and remote directory + * Consumers of Composite directory need not worry whether file is in local or remote + * All such abstractions will be handled by the Composite directory itself + * Implements all required methods by Directory abstraction + */ public class CompositeDirectory extends FilterDirectory { private static final Logger logger = LogManager.getLogger(CompositeDirectory.class); - private final FSDirectory localDirectory; - private final RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter; + private final RemoteSegmentStoreDirectory remoteDirectory; private final FileCache fileCache; - private final AtomicBoolean isRemoteDirectorySet; private final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.WriteLock writeLock = readWriteLock.writeLock(); private final ReentrantReadWriteLock.ReadLock readLock = readWriteLock.readLock(); - public CompositeDirectory(FSDirectory localDirectory, FileCache fileCache) { + /** + * Constructor to initialise the composite directory + * @param localDirectory corresponding to the local FSDirectory + * @param remoteDirectory corresponding to the remote directory + * @param fileCache used to cache the remote files locally + */ + public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirectory remoteDirectory, FileCache fileCache) { super(localDirectory); this.localDirectory = localDirectory; + this.remoteDirectory = remoteDirectory; this.fileCache = fileCache; - this.remoteStoreFileTrackerAdapter = new CompositeDirectoryRemoteStoreFileTrackerAdapter(fileCache); - isRemoteDirectorySet = new AtomicBoolean(false); - } - - public void setRemoteDirectory(Directory remoteDirectory) { - logger.trace("Setting remote Directory ..."); - if (!isRemoteDirectorySet()) { - ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).setRemoteDirectory(remoteDirectory); - isRemoteDirectorySet.set(true); - } } + /** + * Returns names of all files stored in this directory in sorted order + * Does not include locally stored block files (having _block_ in their names) + * + * @throws IOException in case of I/O error + */ @Override public String[] listAll() throws IOException { logger.trace("listAll() called ..."); readLock.lock(); try { - String[] remoteFiles = new String[0]; String[] localFiles = localDirectory.listAll(); - if (isRemoteDirectorySet()) remoteFiles = ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter) - .getRemoteFiles(); - logger.trace("LocalDirectory files : " + Arrays.toString(localFiles)); - logger.trace("Remote Directory files : " + Arrays.toString(remoteFiles)); + logger.trace("LocalDirectory files : {}", () -> Arrays.toString(localFiles)); Set allFiles = new HashSet<>(Arrays.asList(localFiles)); - allFiles.addAll(Arrays.asList(remoteFiles)); - - Set localLuceneFiles = allFiles.stream().filter(file -> !isBlockFile(file)).collect(Collectors.toUnmodifiableSet()); + if (remoteDirectory != null) { + String[] remoteFiles = getRemoteFiles(); + allFiles.addAll(Arrays.asList(remoteFiles)); + logger.trace("Remote Directory files : {}", () -> Arrays.toString(remoteFiles)); + } + Set localLuceneFiles = allFiles.stream() + .filter(file -> !FileType.isBlockFile(file)) + .collect(Collectors.toUnmodifiableSet()); String[] files = new String[localLuceneFiles.size()]; localLuceneFiles.toArray(files); Arrays.sort(files); - - logger.trace("listAll() returns : " + Arrays.toString(files)); - + logger.trace("listAll() returns : {}", () -> Arrays.toString(files)); return files; } finally { readLock.unlock(); } } + /** + * Removes an existing file in the directory. + * Throws {@link NoSuchFileException} or {@link FileNotFoundException} in case file is not present locally and in remote as well + * @param name the name of an existing file. + * @throws IOException in case of I/O error + */ @Override public void deleteFile(String name) throws IOException { - logger.trace("deleteFile() called " + name); + logger.trace("deleteFile() called {}", name); writeLock.lock(); try { localDirectory.deleteFile(name); - remoteStoreFileTrackerAdapter.removeFileFromTracker(name); fileCache.remove(localDirectory.getDirectory().resolve(name)); - logFileTracker(); + } catch (NoSuchFileException | FileNotFoundException e) { + logger.trace("File {} not found in local, trying to delete from Remote", name); + try { + remoteDirectory.deleteFile(name); + } finally { + writeLock.unlock(); + } } finally { writeLock.unlock(); } } + /** + * Returns the byte length of a file in the directory. + * Throws {@link NoSuchFileException} or {@link FileNotFoundException} in case file is not present locally and in remote as well + * @param name the name of an existing file. + * @throws IOException in case of I/O error + */ @Override public long fileLength(String name) throws IOException { - logger.trace("fileLength() called " + name); + logger.trace("fileLength() called {}", name); readLock.lock(); try { - if (remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) { - logger.trace("fileLength from Local " + localDirectory.fileLength(name)); - return localDirectory.fileLength(name); + long fileLength; + if (Arrays.asList(getRemoteFiles()).contains(name) == false) { + fileLength = localDirectory.fileLength(name); + logger.trace("fileLength from Local {}", fileLength); } else { - logger.trace( - "fileLength from Remote " - + ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).getFileLength(name) - ); - return ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).getFileLength(name); + fileLength = remoteDirectory.fileLength(name); + logger.trace("fileLength from Remote {}", fileLength); } + return fileLength; } finally { readLock.unlock(); } } + /** + * Creates a new, empty file in the directory and returns an {@link IndexOutput} instance for + * appending data to this file. + * @param name the name of the file to create. + * @throws IOException in case of I/O error + */ @Override public IndexOutput createOutput(String name, IOContext context) throws IOException { - logger.trace("createOutput() called " + name); + logger.trace("createOutput() called {}", name); writeLock.lock(); try { - remoteStoreFileTrackerAdapter.trackFile(name, FileState.DISK, FileType.NON_BLOCK); - logFileTracker(); return localDirectory.createOutput(name, context); } finally { writeLock.unlock(); } } + /** + * Creates a new, empty, temporary file in the directory and returns an {@link IndexOutput} + * instance for appending data to this file. + * + *

    The temporary file name (accessible via {@link IndexOutput#getName()}) will start with + * {@code prefix}, end with {@code suffix} and have a reserved file extension {@code .tmp}. + */ @Override public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException { - logger.trace("createOutput() called " + prefix + "," + suffix); + logger.trace("createTempOutput() called {} , {}", prefix, suffix); writeLock.lock(); try { return localDirectory.createTempOutput(prefix, suffix, context); @@ -145,22 +177,31 @@ public IndexOutput createTempOutput(String prefix, String suffix, IOContext cont } } + /** + * Ensures that any writes to these files are moved to stable storage (made durable). + * @throws IOException in case of I/O error + */ @Override public void sync(Collection names) throws IOException { - logger.trace("sync() called " + names); + logger.trace("sync() called {}", names); writeLock.lock(); try { Collection newLocalFiles = new ArrayList<>(); + Collection remoteFiles = Arrays.asList(getRemoteFiles()); for (String name : names) { - if (remoteStoreFileTrackerAdapter.getFileState(name).equals(FileState.DISK)) newLocalFiles.add(name); + if (remoteFiles.contains(name) == false) newLocalFiles.add(name); } - logger.trace("Synced files : " + newLocalFiles); + logger.trace("Synced files : {}", newLocalFiles); localDirectory.sync(newLocalFiles); } finally { writeLock.unlock(); } } + /** + * Ensures that directory metadata, such as recent file renames, are moved to stable storage. + * @throws IOException in case of I/O error + */ @Override public void syncMetaData() throws IOException { logger.trace("syncMetaData() called "); @@ -172,108 +213,133 @@ public void syncMetaData() throws IOException { } } + /** + * Renames {@code source} file to {@code dest} file where {@code dest} must not already exist in + * the directory. + * @throws IOException in case of I/O error + */ @Override public void rename(String source, String dest) throws IOException { - logger.trace("rename() called " + source + " -> " + dest); + logger.trace("rename() called {}, {}", source, dest); writeLock.lock(); try { localDirectory.rename(source, dest); - remoteStoreFileTrackerAdapter.trackFile( - dest, - remoteStoreFileTrackerAdapter.getFileState(source), - remoteStoreFileTrackerAdapter.getFileType(source) - ); - remoteStoreFileTrackerAdapter.removeFileFromTracker(source); - logFileTracker(); } finally { writeLock.unlock(); } } + /** + * Opens a stream for reading an existing file. + * Check whether the file is present locally or in remote and return the IndexInput accordingly + * @param name the name of an existing file. + * @throws IOException in case of I/O error + */ @Override public IndexInput openInput(String name, IOContext context) throws IOException { - logger.trace("openInput() called " + name); + logger.trace("openInput() called {}", name); writeLock.lock(); try { - if (!remoteStoreFileTrackerAdapter.isFilePresent(name)) { - // Print filename to check which file is not present in tracker - logger.trace("File not found in tracker"); + if (Arrays.asList(getRemoteFiles()).contains(name) == false) { + // If file has not yet been uploaded to Remote Store, fetch it from the local directory + logger.trace("File found in disk"); return localDirectory.openInput(name, context); + } else { + // If file has been uploaded to the Remote Store, fetch it from the Remote Store in blocks via + // OnDemandCompositeBlockIndexInput + logger.trace("File to be fetched from Remote"); + return new OnDemandCompositeBlockIndexInput(remoteDirectory, name, localDirectory, fileCache, context); } - IndexInput indexInput = null; - switch (remoteStoreFileTrackerAdapter.getFileState(name)) { - case DISK: - logger.trace("File found in disk "); - indexInput = localDirectory.openInput(name, context); - break; - - case REMOTE: - logger.trace("File to be fetched from Remote "); - indexInput = new OnDemandCompositeBlockIndexInput(remoteStoreFileTrackerAdapter, name, localDirectory); - break; - } - return indexInput; } finally { writeLock.unlock(); } } + /** + * Acquires and returns a {@link Lock} for a file with the given name. + * @param name the name of the lock file + * @throws LockObtainFailedException (optional specific exception) if the lock could not be + * obtained because it is currently held elsewhere. + * @throws IOException in case of I/O error + */ @Override public Lock obtainLock(String name) throws IOException { - logger.trace("obtainLock() called " + name); - return localDirectory.obtainLock(name); + logger.trace("obtainLock() called {}", name); + writeLock.lock(); + try { + return localDirectory.obtainLock(name); + } finally { + writeLock.unlock(); + } } + /** + * Closes the directory + * @throws IOException in case of I/O error + */ @Override public void close() throws IOException { - localDirectory.close(); + writeLock.lock(); + try { + localDirectory.close(); + } finally { + writeLock.unlock(); + } } + /** + * Returns a set of files currently pending deletion in this directory. + * @throws IOException in case of I/O error + */ @Override public Set getPendingDeletions() throws IOException { - return localDirectory.getPendingDeletions(); + readLock.lock(); + try { + return localDirectory.getPendingDeletions(); + } finally { + readLock.unlock(); + } } + /** + * Function to perform operations once files have been uploaded to Remote Store + * Currently deleting the local files here, as once uploaded to Remote, local files are safe to delete + * @param files : recent files which have been successfully uploaded to Remote Store + * @throws IOException in case of I/O error + */ public void afterSyncToRemote(Collection files) throws IOException { - logger.trace("afterSyncToRemote called for " + files); - if (!isRemoteDirectorySet()) throw new UnsupportedOperationException( - "Cannot perform afterSyncToRemote if Remote Directory is not set" - ); - List delFiles = new ArrayList<>(); + logger.trace("afterSyncToRemote called for {}", files); + if (remoteDirectory == null) { + logger.trace("afterSyncToRemote called even though remote directory is not set"); + return; + } for (String fileName : files) { - if (isSegmentsOrLockFile(fileName)) continue; writeLock.lock(); try { - if (remoteStoreFileTrackerAdapter.isFilePresent(fileName) - && remoteStoreFileTrackerAdapter.getFileState(fileName).equals(FileState.DISK)) { - remoteStoreFileTrackerAdapter.updateFileState(fileName, FileState.REMOTE); - } + localDirectory.deleteFile(fileName); } finally { writeLock.unlock(); } - localDirectory.deleteFile(fileName); - delFiles.add(fileName); } - logger.trace("Files removed form local " + delFiles); - logFileTracker(); - } - - private boolean isSegmentsOrLockFile(String fileName) { - if (fileName.startsWith("segments_") || fileName.endsWith(".si") || fileName.endsWith(".lock")) return true; - return false; - } - - private boolean isBlockFile(String fileName) { - if (fileName.contains("_block_")) return true; - return false; - } - - private boolean isRemoteDirectorySet() { - return isRemoteDirectorySet.get(); } - public void logFileTracker() { - String res = ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).logFileTracker(); - logger.trace(res); + /** + * Return the list of files present in Remote + */ + private String[] getRemoteFiles() { + String[] remoteFiles; + try { + remoteFiles = remoteDirectory.listAll(); + } catch (Exception e) { + /** + * There are two scenarios where the listAll() call on remote directory fails: + * - When remote directory is not set + * - When init() of remote directory has not yet been called (which results in NullPointerException while calling listAll() for RemoteSegmentStoreDirectory) + * + * Returning an empty list in these scenarios + */ + remoteFiles = new String[0]; + } + return remoteFiles; } } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java deleted file mode 100644 index 8256e22eadbca..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryFactory.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.store; - -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.FSDirectory; -import org.opensearch.index.IndexSettings; -import org.opensearch.index.shard.ShardPath; -import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.plugins.IndexStorePlugin; -import org.opensearch.repositories.RepositoriesService; - -import java.io.IOException; -import java.util.function.Supplier; - -public class CompositeDirectoryFactory implements IndexStorePlugin.DirectoryFactory { - - private final Supplier repositoriesService; - private final FileCache remoteStoreFileCache; - - public CompositeDirectoryFactory(Supplier repositoriesService, FileCache remoteStoreFileCache) { - this.repositoriesService = repositoriesService; - this.remoteStoreFileCache = remoteStoreFileCache; - } - - @Override - public Directory newDirectory(IndexSettings indexSettings, ShardPath shardPath) throws IOException { - final FSDirectory primaryDirectory = FSDirectory.open(shardPath.resolveIndex()); - return new CompositeDirectory(primaryDirectory, remoteStoreFileCache); - } -} diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java deleted file mode 100644 index 675c6b8c39871..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectoryRemoteStoreFileTrackerAdapter.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * 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.store; - -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.IndexInput; -import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.utils.BlobFetchRequest; -import org.opensearch.index.store.remote.utils.TransferManager; -import org.opensearch.index.store.remote.utils.filetracker.FileState; -import org.opensearch.index.store.remote.utils.filetracker.FileTrackingInfo; -import org.opensearch.index.store.remote.utils.filetracker.FileType; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -public class CompositeDirectoryRemoteStoreFileTrackerAdapter implements RemoteStoreFileTrackerAdapter { - - private FileCache fileCache; - private Map fileTracker; - private RemoteSegmentStoreDirectory remoteDirectory; - - public CompositeDirectoryRemoteStoreFileTrackerAdapter(FileCache fileCache) { - this.fileCache = fileCache; - remoteDirectory = null; - this.fileTracker = new HashMap<>(); - } - - public void setRemoteDirectory(Directory remoteDirectory) { - this.remoteDirectory = (RemoteSegmentStoreDirectory) remoteDirectory; - } - - public String getUploadedFileName(String name) { - return remoteDirectory.getExistingRemoteFilename(name); - } - - public long getFileLength(String name) { - try { - return remoteDirectory.fileLength(name); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOException { - return new TransferManager(remoteDirectory.getDataDirectoryBlobContainer(), fileCache).fetchBlob(blobFetchRequest); - } - - public void trackFile(String name, FileState fileState, FileType fileType) { - if (!fileTracker.containsKey(name)) { - fileTracker.put(name, new FileTrackingInfo(fileState, fileType)); - } - } - - public void updateFileType(String name, FileType fileType) { - FileTrackingInfo fileTrackingInfo = fileTracker.get(name); - if (fileTrackingInfo != null) { - fileTracker.put(name, new FileTrackingInfo(fileTrackingInfo.getFileState(), fileType)); - } - } - - public void updateFileState(String name, FileState fileState) { - FileTrackingInfo fileTrackingInfo = fileTracker.get(name); - if (fileTrackingInfo != null) { - fileTracker.put(name, new FileTrackingInfo(fileState, fileTrackingInfo.getFileType())); - } - } - - public void removeFileFromTracker(String name) { - fileTracker.remove(name); - } - - public FileState getFileState(String name) { - if (!fileTracker.containsKey(name)) { - return null; - } - return fileTracker.get(name).getFileState(); - } - - public FileType getFileType(String name) { - if (!fileTracker.containsKey(name)) { - return null; - } - return fileTracker.get(name).getFileType(); - } - - public boolean isFilePresent(String name) { - return fileTracker.containsKey(name); - } - - public String[] getRemoteFiles() throws IOException { - String[] remoteFiles; - try { - remoteFiles = remoteDirectory.listAll(); - } catch (Exception e) { - remoteFiles = new String[0]; - } - return remoteFiles; - } - - public String logFileTracker() { - String result = ""; - for (Map.Entry entry : fileTracker.entrySet()) { - result += entry.getKey() - + " : " - + entry.getValue().getFileType().name() - + " , " - + entry.getValue().getFileState().name() - + "\n"; - } - return result; - } -} diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index 99f78130ad3ef..c94eafe4b4661 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -28,9 +28,11 @@ import org.opensearch.common.blobstore.transfer.RemoteTransferContainer; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeIndexInputStream; import org.opensearch.common.blobstore.transfer.stream.OffsetRangeInputStream; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.unit.ByteSizeUnit; import org.opensearch.index.store.exception.ChecksumCombinationException; +import org.opensearch.index.store.remote.utils.BlockIOContext; import java.io.FileNotFoundException; import java.io.IOException; @@ -203,10 +205,18 @@ public IndexInput openInput(String name, IOContext context) throws IOException { public IndexInput openInput(String name, long fileLength, IOContext context) throws IOException { InputStream inputStream = null; try { - inputStream = blobContainer.readBlob(name); - return new RemoteIndexInput(name, downloadRateLimiter.apply(inputStream), fileLength); + if (context instanceof BlockIOContext) { + long position = ((BlockIOContext) context).getBlockStart(); + long length = ((BlockIOContext) context).getBlockSize(); + inputStream = blobContainer.readBlob(name, position, length); + byte[] bytes = downloadRateLimiter.apply(inputStream).readAllBytes(); + return new ByteArrayIndexInput(name, bytes); + } else { + inputStream = blobContainer.readBlob(name); + return new RemoteIndexInput(name, downloadRateLimiter.apply(inputStream), fileLength); + } } catch (Exception e) { - // Incase the RemoteIndexInput creation fails, close the input stream to avoid file handler leak. + // In case the RemoteIndexInput creation fails, close the input stream to avoid file handler leak. if (inputStream != null) { try { inputStream.close(); 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 868d1803de152..8c0ecb4cc783a 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteSegmentStoreDirectory.java @@ -25,7 +25,6 @@ import org.apache.lucene.util.Version; import org.opensearch.common.UUIDs; import org.opensearch.common.annotation.PublicApi; -import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.common.collect.Tuple; import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.logging.Loggers; @@ -137,10 +136,6 @@ public RemoteSegmentStoreDirectory( init(); } - public BlobContainer getDataDirectoryBlobContainer() { - return remoteDataDirectory.getBlobContainer(); - } - /** * 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 @@ -703,7 +698,7 @@ private String getChecksumOfLocalFile(Directory directory, String file) throws I } } - public String getExistingRemoteFilename(String localFilename) { + private String getExistingRemoteFilename(String localFilename) { if (segmentsUploadedToRemoteStore.containsKey(localFilename)) { return segmentsUploadedToRemoteStore.get(localFilename).uploadedFilename; } else { diff --git a/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java b/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java deleted file mode 100644 index 29f05c8dc60e8..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/RemoteStoreFileTrackerAdapter.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.store; - -import org.apache.lucene.store.IndexInput; -import org.opensearch.index.store.remote.utils.BlobFetchRequest; -import org.opensearch.index.store.remote.utils.filetracker.FileState; -import org.opensearch.index.store.remote.utils.filetracker.FileType; - -import java.io.IOException; - -public interface RemoteStoreFileTrackerAdapter { - IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOException; - - void trackFile(String name, FileState fileState, FileType fileType); - - void updateFileType(String name, FileType fileType); - - void updateFileState(String name, FileState fileState); - - void removeFileFromTracker(String name); - - FileState getFileState(String name); - - FileType getFileType(String name); - - boolean isFilePresent(String name); -} diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java index 8792e2294a81b..b2af3e7305c5d 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java @@ -11,97 +11,122 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.opensearch.index.store.CompositeDirectoryRemoteStoreFileTrackerAdapter; -import org.opensearch.index.store.RemoteStoreFileTrackerAdapter; -import org.opensearch.index.store.remote.utils.BlobFetchRequest; +import org.opensearch.index.store.RemoteSegmentStoreDirectory; +import org.opensearch.index.store.remote.filecache.CachedIndexInput; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.utils.BlockIOContext; +import java.io.BufferedOutputStream; +import java.io.FileNotFoundException; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicBoolean; +/** + * OnDemandCompositeBlockIndexInput is used by the Composite Directory to read data in blocks from Remote and cache those blocks in FileCache + */ public class OnDemandCompositeBlockIndexInput extends OnDemandBlockIndexInput { private static final Logger logger = LogManager.getLogger(OnDemandCompositeBlockIndexInput.class); - private final RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter; + private final RemoteSegmentStoreDirectory remoteDirectory; private final String fileName; private final Long originalFileSize; - private final FSDirectory directory; + private final FSDirectory localDirectory; + private final IOContext context; + private final FileCache fileCache; public OnDemandCompositeBlockIndexInput( - RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, + RemoteSegmentStoreDirectory remoteDirectory, String fileName, - FSDirectory directory - ) { + FSDirectory localDirectory, + FileCache fileCache, + IOContext context + ) throws IOException { this( OnDemandBlockIndexInput.builder() .resourceDescription("OnDemandCompositeBlockIndexInput") .isClone(false) .offset(0L) - .length(getFileLength(remoteStoreFileTrackerAdapter, fileName)), - remoteStoreFileTrackerAdapter, + .length(remoteDirectory.fileLength(fileName)), + remoteDirectory, fileName, - directory + localDirectory, + fileCache, + context ); } public OnDemandCompositeBlockIndexInput( Builder builder, - RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, + RemoteSegmentStoreDirectory remoteDirectory, String fileName, - FSDirectory directory - ) { + FSDirectory localDirectory, + FileCache fileCache, + IOContext context + ) throws IOException { super(builder); - this.remoteStoreFileTrackerAdapter = remoteStoreFileTrackerAdapter; - this.directory = directory; + this.remoteDirectory = remoteDirectory; + this.localDirectory = localDirectory; this.fileName = fileName; - originalFileSize = getFileLength(remoteStoreFileTrackerAdapter, fileName); + this.fileCache = fileCache; + this.context = context; + originalFileSize = remoteDirectory.fileLength(fileName); } @Override protected OnDemandCompositeBlockIndexInput buildSlice(String sliceDescription, long offset, long length) { - return new OnDemandCompositeBlockIndexInput( - OnDemandBlockIndexInput.builder() - .blockSizeShift(blockSizeShift) - .isClone(true) - .offset(this.offset + offset) - .length(length) - .resourceDescription(sliceDescription), - remoteStoreFileTrackerAdapter, - fileName, - directory - ); + try { + return new OnDemandCompositeBlockIndexInput( + OnDemandBlockIndexInput.builder() + .blockSizeShift(blockSizeShift) + .isClone(true) + .offset(this.offset + offset) + .length(length) + .resourceDescription(sliceDescription), + remoteDirectory, + fileName, + localDirectory, + fileCache, + context + ); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Override protected IndexInput fetchBlock(int blockId) throws IOException { - logger.trace("fetchBlock called with blockId -> " + blockId); - final String uploadedFileName = ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter) - .getUploadedFileName(fileName); + logger.trace("fetchBlock called with blockId -> {}", blockId); final String blockFileName = fileName + "_block_" + blockId; final long blockStart = getBlockStart(blockId); final long length = getActualBlockSize(blockId); logger.trace( - "File: " - + uploadedFileName - + ", Block File: " - + blockFileName - + ", BlockStart: " - + blockStart - + ", Length: " - + length - + ", BlockSize: " - + blockSize - + ", OriginalFileSize: " - + originalFileSize + "File: {} , Block File: {} , Length: {} , BlockSize: {} , OriginalFileSize: {}", + fileName, + blockFileName, + blockStart, + length, + originalFileSize ); - BlobFetchRequest.BlobPart blobPart = new BlobFetchRequest.BlobPart(uploadedFileName, blockStart, length); - BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() - .blobParts(new ArrayList<>(Arrays.asList(blobPart))) - .directory(directory) - .fileName(blockFileName) - .build(); - return remoteStoreFileTrackerAdapter.fetchBlob(blobFetchRequest); + Path blockFilePath = getLocalFilePath(blockFileName); + final CachedIndexInput cacheEntry = fileCache.compute(blockFilePath, (path, cachedIndexInput) -> { + if (cachedIndexInput == null || cachedIndexInput.isClosed()) { + // Doesn't exist or is closed, either way create a new one + IndexInput indexInput = fetchIndexInput(blockFileName, blockStart, length); + return new CachedIndexInputImpl(indexInput); + } else { + logger.trace("Block already present in cache"); + // already in the cache and ready to be used (open) + return cachedIndexInput; + } + }); + + return cacheEntry.getIndexInput(); } @Override @@ -116,7 +141,89 @@ private long getActualBlockSize(int blockId) { return (blockId != getBlock(originalFileSize - 1)) ? blockSize : getBlockOffset(originalFileSize - 1) + 1; } - private static long getFileLength(RemoteStoreFileTrackerAdapter remoteStoreFileTrackerAdapter, String fileName) { - return ((CompositeDirectoryRemoteStoreFileTrackerAdapter) remoteStoreFileTrackerAdapter).getFileLength(fileName); + private Path getLocalFilePath(String file) { + return localDirectory.getDirectory().resolve(file); + } + + private IndexInput fetchIndexInput(String blockFileName, long start, long length) { + IndexInput indexInput; + Path filePath = getLocalFilePath(blockFileName); + try { + // Fetch from local if block file is present locally in disk + indexInput = localDirectory.openInput(blockFileName, IOContext.READ); + logger.trace("Block file present locally, just putting it in cache"); + } catch (FileNotFoundException | NoSuchFileException e) { + logger.trace("Block file not present locally, fetching from Remote"); + // If block file is not present locally in disk, fetch from remote and persist the block file in disk + try ( + OutputStream fileOutputStream = Files.newOutputStream(filePath); + OutputStream localFileOutputStream = new BufferedOutputStream(fileOutputStream) + ) { + logger.trace("Fetching block file from Remote"); + indexInput = remoteDirectory.openInput(fileName, new BlockIOContext(IOContext.READ, start, length)); + logger.trace("Persisting the fetched blocked file from Remote"); + int indexInputLength = (int) indexInput.length(); + byte[] bytes = new byte[indexInputLength]; + indexInput.readBytes(bytes, 0, indexInputLength); + localFileOutputStream.write(bytes); + } catch (Exception err) { + logger.trace("Exception while fetching block from remote and persisting it on disk"); + throw new RuntimeException(err); + } + } catch (Exception e) { + logger.trace("Exception while fetching block file locally"); + throw new RuntimeException(e); + } + return indexInput; + } + + /** + * Implementation of the CachedIndexInput interface + */ + private class CachedIndexInputImpl implements CachedIndexInput { + + IndexInput indexInput; + AtomicBoolean isClosed; + + /** + * Constructor - takes IndexInput as parameter + */ + CachedIndexInputImpl(IndexInput indexInput) { + this.indexInput = indexInput; + isClosed = new AtomicBoolean(false); + } + + /** + * Returns the wrapped indexInput + */ + @Override + public IndexInput getIndexInput() throws IOException { + return indexInput; + } + + /** + * Returns the length of the wrapped indexInput + */ + @Override + public long length() { + return indexInput.length(); + } + + /** + * Checks if the wrapped indexInput is closed + */ + @Override + public boolean isClosed() { + return isClosed.get(); + } + + /** + * Closes the wrapped indexInput + */ + @Override + public void close() throws Exception { + indexInput.close(); + isClosed.set(true); + } } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java b/server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java new file mode 100644 index 0000000000000..da94e4a46d307 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java @@ -0,0 +1,62 @@ +/* + * 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.store.remote.utils; + +import org.apache.lucene.store.IOContext; + +/** + * BlockIOContext is an extension of IOContext which can be used to pass block related information to the openInput() method of any directory + */ +public class BlockIOContext extends IOContext { + + private final boolean isBlockRequest; + private long blockStart; + private long blockSize; + + /** + * Default constructor + */ + BlockIOContext(IOContext ctx) { + super(ctx.context); + this.isBlockRequest = false; + this.blockStart = -1; + this.blockSize = -1; + } + + /** + * Constructor to initialise BlockIOContext with block related information + */ + public BlockIOContext(IOContext ctx, long blockStart, long blockSize) { + super(ctx.context); + this.isBlockRequest = true; + this.blockStart = blockStart; + this.blockSize = blockSize; + } + + /** + * Function to check if the Context contains a block request or not + */ + public boolean isBlockRequest() { + return isBlockRequest; + } + + /** + * Getter for blockStart + */ + public long getBlockStart() { + return blockStart; + } + + /** + * Getter for blockSize + */ + public long getBlockSize() { + return blockSize; + } +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java b/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java new file mode 100644 index 0000000000000..418f8a24a5f24 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java @@ -0,0 +1,38 @@ +/* + * 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.store.remote.utils; + +/** + * Enum to represent whether a file is block or not + */ +public enum FileType { + /** + * Block file + */ + BLOCK, + /** + * Non block file + */ + NON_BLOCK; + + /** + * Returns if the fileType is a block file or not + */ + public static boolean isBlockFile(FileType fileType) { + return fileType.equals(FileType.BLOCK); + } + + /** + * Returns if the fileName is block file or not + */ + public static boolean isBlockFile(String fileName) { + if (fileName.contains("_block_")) return true; + return false; + } +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java deleted file mode 100644 index 73b39bac35f35..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileState.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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.store.remote.utils.filetracker; - -public enum FileState { - /** - * DISK State means that currently the file is present only locally and has not yet been uploaded to the Remote Store - */ - DISK, - /** - * REMOTE State means that the file has been successfully uploaded to the Remote Store and is safe to be removed locally - */ - REMOTE; -} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java deleted file mode 100644 index 60310b0fc39a2..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileTrackingInfo.java +++ /dev/null @@ -1,27 +0,0 @@ -/* - * 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.store.remote.utils.filetracker; - -public class FileTrackingInfo { - private final FileState fileState; - private final FileType fileType; - - public FileTrackingInfo(FileState fileState, FileType fileType) { - this.fileState = fileState; - this.fileType = fileType; - } - - public FileState getFileState() { - return fileState; - } - - public FileType getFileType() { - return fileType; - } -} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java b/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java deleted file mode 100644 index 18506f7062b7c..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/filetracker/FileType.java +++ /dev/null @@ -1,18 +0,0 @@ -/* - * 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.store.remote.utils.filetracker; - -public enum FileType { - BLOCK, - NON_BLOCK; - - public boolean isBlockFile(FileType fileType) { - return fileType.equals(FileType.BLOCK); - } -} diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index 73b4cf3259d5b..aef40b80ea636 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -136,6 +136,7 @@ import org.opensearch.index.shard.IndexingOperationListener; import org.opensearch.index.shard.IndexingStats; import org.opensearch.index.shard.IndexingStats.Stats.DocStatusStats; +import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.translog.InternalTranslogFactory; import org.opensearch.index.translog.RemoteBlobStoreInternalTranslogFactory; import org.opensearch.index.translog.TranslogFactory; @@ -354,6 +355,7 @@ public class IndicesService extends AbstractLifecycleComponent private final BiFunction translogFactorySupplier; private volatile TimeValue clusterDefaultRefreshInterval; private final SearchRequestStats searchRequestStats; + private final FileCache fileCache; @Override protected void doStart() { @@ -388,7 +390,8 @@ public IndicesService( @Nullable RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, RecoverySettings recoverySettings, CacheService cacheService, - RemoteStoreSettings remoteStoreSettings + RemoteStoreSettings remoteStoreSettings, + FileCache fileCache ) { this.settings = settings; this.threadPool = threadPool; @@ -440,6 +443,7 @@ public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, lon this.directoryFactories = directoryFactories; this.recoveryStateFactories = recoveryStateFactories; + this.fileCache = fileCache; // doClose() is called when shutting down a node, yet there might still be ongoing requests // that we need to wait for before closing some resources such as the caches. In order to // avoid closing these resources while ongoing requests are still being processed, we use a @@ -873,7 +877,8 @@ private synchronized IndexService createIndexService( directoryFactories, () -> allowExpensiveQueries, indexNameExpressionResolver, - recoveryStateFactories + recoveryStateFactories, + fileCache ); for (IndexingOperationListener operationListener : indexingOperationListeners) { indexModule.addIndexOperationListener(operationListener); @@ -963,7 +968,8 @@ public synchronized MapperService createIndexMapperService(IndexMetadata indexMe directoryFactories, () -> allowExpensiveQueries, indexNameExpressionResolver, - recoveryStateFactories + recoveryStateFactories, + fileCache ); pluginsService.onIndexModule(indexModule); return indexModule.newIndexMapperService(xContentRegistry, mapperRegistry, scriptService); diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index b24d342fb616b..d42cb21be36dc 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -869,7 +869,8 @@ protected Node( remoteStoreStatsTrackerFactory, recoverySettings, cacheService, - remoteStoreSettings + remoteStoreSettings, + fileCache ); final IngestService ingestService = new IngestService( diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 4ce4936c047d9..6eddd8c4a9b4a 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -278,7 +278,8 @@ public void testWrapperIsBound() throws IOException { Collections.emptyMap(), () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - Collections.emptyMap() + Collections.emptyMap(), + null ); module.setReaderWrapper(s -> new Wrapper()); @@ -304,7 +305,8 @@ public void testRegisterIndexStore() throws IOException { indexStoreFactories, () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - Collections.emptyMap() + Collections.emptyMap(), + null ); final IndexService indexService = newIndexService(module); @@ -632,7 +634,8 @@ public void testRegisterCustomRecoveryStateFactory() throws IOException { Collections.emptyMap(), () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - recoveryStateFactories + recoveryStateFactories, + null ); final IndexService indexService = newIndexService(module); @@ -664,7 +667,8 @@ private static IndexModule createIndexModule(IndexSettings indexSettings, Analys Collections.emptyMap(), () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - Collections.emptyMap() + Collections.emptyMap(), + null ); } diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index 9c58fc8fde084..b035e117ff3ee 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2078,7 +2078,8 @@ public void onFailure(final Exception e) { new RemoteStoreStatsTrackerFactory(clusterService, settings), DefaultRecoverySettings.INSTANCE, new CacheModule(new ArrayList<>(), settings).getCacheService(), - DefaultRemoteStoreSettings.INSTANCE + DefaultRemoteStoreSettings.INSTANCE, + null ); final RecoverySettings recoverySettings = new RecoverySettings(settings, clusterSettings); snapshotShardsService = new SnapshotShardsService( From 932ad92db9bb4bbba5c8cdf943c67b13234968af Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Tue, 30 Apr 2024 15:26:05 +0530 Subject: [PATCH 07/22] Modify TransferManager - replace BlobContainer with Functional Interface to fetch an InputStream instead Signed-off-by: Shreyansh Ray --- .../remotestore/CompositeDirectoryIT.java | 21 ++- .../index/store/CompositeDirectory.java | 103 +++++++++++---- .../index/store/RemoteDirectory.java | 2 + .../RemoteSnapshotDirectoryFactory.java | 2 +- .../OnDemandCompositeBlockIndexInput.java | 122 +++--------------- .../filecache/WrappedCachedIndexInput.java | 66 ++++++++++ .../store/remote/utils/TransferManager.java | 26 ++-- .../remote/utils/TransferManagerTests.java | 2 +- 8 files changed, 200 insertions(+), 144 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/store/remote/filecache/WrappedCachedIndexInput.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java index 35499c2f8cc5e..71255236a3809 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java @@ -8,6 +8,8 @@ package org.opensearch.remotestore; +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; import org.opensearch.action.admin.indices.get.GetIndexRequest; @@ -19,8 +21,11 @@ import org.opensearch.index.IndexService; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.CompositeDirectory; +import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; import org.opensearch.indices.IndicesService; import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.junit.annotations.TestLogging; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; @@ -28,11 +33,21 @@ import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +@ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST) +//@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") public class CompositeDirectoryIT extends RemoteStoreBaseIntegTestCase { + + @Override + protected Settings featureFlagSettings() { + Settings.Builder featureSettings = Settings.builder(); + featureSettings.put(FeatureFlags.WRITEABLE_REMOTE_INDEX, true); + + return featureSettings.build(); + } + public void testCompositeDirectory() throws Exception { Settings settings = Settings.builder() - .put(super.featureFlagSettings()) - .put(FeatureFlags.WRITEABLE_REMOTE_INDEX, "true") .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), "partial") @@ -60,7 +75,7 @@ public void testCompositeDirectory() throws Exception { assertEquals("true", indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); assertEquals(REPOSITORY_NAME, indexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); assertEquals(REPOSITORY_2_NAME, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); - assertEquals("partial", indexSettings.get("index.store.locality")); + assertEquals("partial", indexSettings.get("index.store.data_locality")); ensureGreen("test-idx-1"); indexData(10, false, "test-idx-1"); diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index e668ac31e57ee..26faf9d2039c0 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -17,13 +17,20 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; +import org.opensearch.common.lucene.store.FilterIndexOutput; +import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.index.store.remote.file.OnDemandCompositeBlockIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.filecache.WrappedCachedIndexInput; +import org.opensearch.index.store.remote.utils.BlobFetchRequest; +import org.opensearch.index.store.remote.utils.BlockIOContext; import org.opensearch.index.store.remote.utils.FileType; +import org.opensearch.index.store.remote.utils.TransferManager; import java.io.FileNotFoundException; import java.io.IOException; import java.nio.file.NoSuchFileException; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -43,6 +50,7 @@ public class CompositeDirectory extends FilterDirectory { private final FSDirectory localDirectory; private final RemoteSegmentStoreDirectory remoteDirectory; private final FileCache fileCache; + private final TransferManager transferManager; private final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.WriteLock writeLock = readWriteLock.writeLock(); private final ReentrantReadWriteLock.ReadLock readLock = readWriteLock.readLock(); @@ -58,6 +66,9 @@ public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirector this.localDirectory = localDirectory; this.remoteDirectory = remoteDirectory; this.fileCache = fileCache; + transferManager = new TransferManager( + (name, position, length) -> new InputStreamIndexInput(remoteDirectory.openInput(name, new BlockIOContext(IOContext.DEFAULT, position, length)), length), + fileCache); } /** @@ -107,11 +118,7 @@ public void deleteFile(String name) throws IOException { fileCache.remove(localDirectory.getDirectory().resolve(name)); } catch (NoSuchFileException | FileNotFoundException e) { logger.trace("File {} not found in local, trying to delete from Remote", name); - try { - remoteDirectory.deleteFile(name); - } finally { - writeLock.unlock(); - } + remoteDirectory.deleteFile(name); } finally { writeLock.unlock(); } @@ -129,7 +136,7 @@ public long fileLength(String name) throws IOException { readLock.lock(); try { long fileLength; - if (Arrays.asList(getRemoteFiles()).contains(name) == false) { + if (isTempFile(name) || fileCache.get(localDirectory.getDirectory().resolve(name)) != null) { fileLength = localDirectory.fileLength(name); logger.trace("fileLength from Local {}", fileLength); } else { @@ -153,7 +160,10 @@ public IndexOutput createOutput(String name, IOContext context) throws IOExcepti logger.trace("createOutput() called {}", name); writeLock.lock(); try { - return localDirectory.createOutput(name, context); + /** + * The CacheableIndexOutput will ensure that the file is added to FileCache once write is completed on this file + */ + return new CacheableIndexOutput(localDirectory.createOutput(name, context), name); } finally { writeLock.unlock(); } @@ -224,6 +234,8 @@ public void rename(String source, String dest) throws IOException { writeLock.lock(); try { localDirectory.rename(source, dest); + fileCache.remove(localDirectory.getDirectory().resolve(source)); + cacheFile(dest); } finally { writeLock.unlock(); } @@ -240,14 +252,31 @@ public IndexInput openInput(String name, IOContext context) throws IOException { logger.trace("openInput() called {}", name); writeLock.lock(); try { - if (Arrays.asList(getRemoteFiles()).contains(name) == false) { - // If file has not yet been uploaded to Remote Store, fetch it from the local directory - logger.trace("File found in disk"); + /** + * We aren't tracking temporary files (created via createTempOutput) currently in FileCache as these are created and then deleted within a very short span of time + * We will be reading them directory from the local directory + */ + if (isTempFile(name)) { return localDirectory.openInput(name, context); - } else { - // If file has been uploaded to the Remote Store, fetch it from the Remote Store in blocks via - // OnDemandCompositeBlockIndexInput - logger.trace("File to be fetched from Remote"); + } + /** + * Return directly from the FileCache (via TransferManager) if complete file is present + */ + else if (fileCache.get(localDirectory.getDirectory().resolve(name)) != null) { + logger.trace("Complete file found in FileCache"); + BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() + .directory(localDirectory) + .fileName(name) + // position and length are not required here since this is a complete file, just adding dummy values for validation + .blobParts(new ArrayList<>(Arrays.asList(new BlobFetchRequest.BlobPart(name, 0, 1)))) + .build(); + return transferManager.fetchBlob(blobFetchRequest); + } + /** + * If file has been uploaded to the Remote Store, fetch it from the Remote Store in blocks via OnDemandCompositeBlockIndexInput + */ + else { + logger.trace("Complete file not in FileCache, to be fetched in Blocks from Remote"); return new OnDemandCompositeBlockIndexInput(remoteDirectory, name, localDirectory, fileCache, context); } } finally { @@ -293,17 +322,17 @@ public void close() throws IOException { */ @Override public Set getPendingDeletions() throws IOException { - readLock.lock(); + writeLock.lock(); try { return localDirectory.getPendingDeletions(); } finally { - readLock.unlock(); + writeLock.unlock(); } } /** * Function to perform operations once files have been uploaded to Remote Store - * Currently deleting the local files here, as once uploaded to Remote, local files are safe to delete + * Currently deleting the local files here, as once uploaded to Remote, local files become eligible for eviction from FileCache * @param files : recent files which have been successfully uploaded to Remote Store * @throws IOException in case of I/O error */ @@ -316,25 +345,34 @@ public void afterSyncToRemote(Collection files) throws IOException { for (String fileName : files) { writeLock.lock(); try { - localDirectory.deleteFile(fileName); + /** + * TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache + * Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality + */ + logger.trace("File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", fileName); + // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); } finally { writeLock.unlock(); } } } + private boolean isTempFile(String name) { + return name.endsWith(".tmp"); + } + /** * Return the list of files present in Remote */ - private String[] getRemoteFiles() { + private String[] getRemoteFiles() throws IOException { String[] remoteFiles; try { remoteFiles = remoteDirectory.listAll(); - } catch (Exception e) { + } catch (NullPointerException e) { /** - * There are two scenarios where the listAll() call on remote directory fails: + * There are two scenarios where the listAll() call on remote directory returns NullPointerException: * - When remote directory is not set - * - When init() of remote directory has not yet been called (which results in NullPointerException while calling listAll() for RemoteSegmentStoreDirectory) + * - When init() of remote directory has not yet been called * * Returning an empty list in these scenarios */ @@ -342,4 +380,25 @@ private String[] getRemoteFiles() { } return remoteFiles; } + + private void cacheFile(String name) throws IOException { + Path filePath = localDirectory.getDirectory().resolve(name); + fileCache.put(filePath, new WrappedCachedIndexInput(localDirectory.openInput(name, IOContext.READ))); + } + + private class CacheableIndexOutput extends FilterIndexOutput { + + String fileName; + + public CacheableIndexOutput(IndexOutput out, String fileName) { + super("CacheableIndexOutput for file : " + fileName, out); + this.fileName = fileName; + } + + @Override + public void close() throws IOException { + super.close(); + cacheFile(fileName); + } + } } diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index c94eafe4b4661..1dfbb3dcc0f36 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -209,7 +209,9 @@ public IndexInput openInput(String name, long fileLength, IOContext context) thr long position = ((BlockIOContext) context).getBlockStart(); long length = ((BlockIOContext) context).getBlockSize(); inputStream = blobContainer.readBlob(name, position, length); + // TODO - Explore how we can buffer small chunks of data instead of having the whole 8MB block in memory byte[] bytes = downloadRateLimiter.apply(inputStream).readAllBytes(); + inputStream.close(); return new ByteArrayIndexInput(name, bytes); } else { inputStream = blobContainer.readBlob(name); diff --git a/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java b/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java index 7cfa738e75e52..177f0526e7571 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java +++ b/server/src/main/java/org/opensearch/index/store/remote/directory/RemoteSnapshotDirectoryFactory.java @@ -94,7 +94,7 @@ private Future createRemoteSnapshotDirectoryFromSnapsho assert indexShardSnapshot instanceof BlobStoreIndexShardSnapshot : "indexShardSnapshot should be an instance of BlobStoreIndexShardSnapshot"; final BlobStoreIndexShardSnapshot snapshot = (BlobStoreIndexShardSnapshot) indexShardSnapshot; - TransferManager transferManager = new TransferManager(blobContainer, remoteStoreFileCache); + TransferManager transferManager = new TransferManager(blobContainer::readBlob, remoteStoreFileCache); return new RemoteSnapshotDirectory(snapshot, localStoreDir, transferManager); }); } diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java index b2af3e7305c5d..f024592330512 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java @@ -13,19 +13,16 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; +import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.index.store.RemoteSegmentStoreDirectory; -import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.utils.BlobFetchRequest; import org.opensearch.index.store.remote.utils.BlockIOContext; +import org.opensearch.index.store.remote.utils.TransferManager; -import java.io.BufferedOutputStream; -import java.io.FileNotFoundException; import java.io.IOException; -import java.io.OutputStream; -import java.nio.file.Files; -import java.nio.file.NoSuchFileException; -import java.nio.file.Path; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.ArrayList; +import java.util.List; /** * OnDemandCompositeBlockIndexInput is used by the Composite Directory to read data in blocks from Remote and cache those blocks in FileCache @@ -39,6 +36,7 @@ public class OnDemandCompositeBlockIndexInput extends OnDemandBlockIndexInput { private final FSDirectory localDirectory; private final IOContext context; private final FileCache fileCache; + private final TransferManager transferManager; public OnDemandCompositeBlockIndexInput( RemoteSegmentStoreDirectory remoteDirectory, @@ -75,6 +73,9 @@ public OnDemandCompositeBlockIndexInput( this.fileName = fileName; this.fileCache = fileCache; this.context = context; + this.transferManager = new TransferManager( + (name, position, length) -> new InputStreamIndexInput(remoteDirectory.openInput(name, new BlockIOContext(context, position, length)), length), + fileCache); originalFileSize = remoteDirectory.fileLength(fileName); } @@ -113,20 +114,12 @@ protected IndexInput fetchBlock(int blockId) throws IOException { length, originalFileSize ); - Path blockFilePath = getLocalFilePath(blockFileName); - final CachedIndexInput cacheEntry = fileCache.compute(blockFilePath, (path, cachedIndexInput) -> { - if (cachedIndexInput == null || cachedIndexInput.isClosed()) { - // Doesn't exist or is closed, either way create a new one - IndexInput indexInput = fetchIndexInput(blockFileName, blockStart, length); - return new CachedIndexInputImpl(indexInput); - } else { - logger.trace("Block already present in cache"); - // already in the cache and ready to be used (open) - return cachedIndexInput; - } - }); - - return cacheEntry.getIndexInput(); + BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() + .directory(localDirectory) + .fileName(blockFileName) + .blobParts(new ArrayList<>(List.of(new BlobFetchRequest.BlobPart(fileName, blockStart, length)))) + .build(); + return transferManager.fetchBlob(blobFetchRequest); } @Override @@ -141,89 +134,4 @@ private long getActualBlockSize(int blockId) { return (blockId != getBlock(originalFileSize - 1)) ? blockSize : getBlockOffset(originalFileSize - 1) + 1; } - private Path getLocalFilePath(String file) { - return localDirectory.getDirectory().resolve(file); - } - - private IndexInput fetchIndexInput(String blockFileName, long start, long length) { - IndexInput indexInput; - Path filePath = getLocalFilePath(blockFileName); - try { - // Fetch from local if block file is present locally in disk - indexInput = localDirectory.openInput(blockFileName, IOContext.READ); - logger.trace("Block file present locally, just putting it in cache"); - } catch (FileNotFoundException | NoSuchFileException e) { - logger.trace("Block file not present locally, fetching from Remote"); - // If block file is not present locally in disk, fetch from remote and persist the block file in disk - try ( - OutputStream fileOutputStream = Files.newOutputStream(filePath); - OutputStream localFileOutputStream = new BufferedOutputStream(fileOutputStream) - ) { - logger.trace("Fetching block file from Remote"); - indexInput = remoteDirectory.openInput(fileName, new BlockIOContext(IOContext.READ, start, length)); - logger.trace("Persisting the fetched blocked file from Remote"); - int indexInputLength = (int) indexInput.length(); - byte[] bytes = new byte[indexInputLength]; - indexInput.readBytes(bytes, 0, indexInputLength); - localFileOutputStream.write(bytes); - } catch (Exception err) { - logger.trace("Exception while fetching block from remote and persisting it on disk"); - throw new RuntimeException(err); - } - } catch (Exception e) { - logger.trace("Exception while fetching block file locally"); - throw new RuntimeException(e); - } - return indexInput; - } - - /** - * Implementation of the CachedIndexInput interface - */ - private class CachedIndexInputImpl implements CachedIndexInput { - - IndexInput indexInput; - AtomicBoolean isClosed; - - /** - * Constructor - takes IndexInput as parameter - */ - CachedIndexInputImpl(IndexInput indexInput) { - this.indexInput = indexInput; - isClosed = new AtomicBoolean(false); - } - - /** - * Returns the wrapped indexInput - */ - @Override - public IndexInput getIndexInput() throws IOException { - return indexInput; - } - - /** - * Returns the length of the wrapped indexInput - */ - @Override - public long length() { - return indexInput.length(); - } - - /** - * Checks if the wrapped indexInput is closed - */ - @Override - public boolean isClosed() { - return isClosed.get(); - } - - /** - * Closes the wrapped indexInput - */ - @Override - public void close() throws Exception { - indexInput.close(); - isClosed.set(true); - } - } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/WrappedCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/WrappedCachedIndexInput.java new file mode 100644 index 0000000000000..67b12a10810ed --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/WrappedCachedIndexInput.java @@ -0,0 +1,66 @@ +/* + * 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.store.remote.filecache; + +import org.apache.lucene.store.IndexInput; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Wrapper implementation of the CachedIndexInput which takes in an IndexInput as parameter + */ +public class WrappedCachedIndexInput implements CachedIndexInput { + + IndexInput indexInput; + AtomicBoolean isClosed; + + /** + * Constructor - takes IndexInput as parameter + */ + public WrappedCachedIndexInput(IndexInput indexInput) { + this.indexInput = indexInput; + isClosed = new AtomicBoolean(false); + } + + /** + * Returns the wrapped indexInput + */ + @Override + public IndexInput getIndexInput() throws IOException { + return indexInput; + } + + /** + * Returns the length of the wrapped indexInput + */ + @Override + public long length() { + return indexInput.length(); + } + + /** + * Checks if the wrapped indexInput is closed + */ + @Override + public boolean isClosed() { + return isClosed.get(); + } + + /** + * Closes the wrapped indexInput + */ + @Override + public void close() throws Exception { + if (!isClosed()) { + indexInput.close(); + isClosed.set(true); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java index 98cad7bfadb09..557a71abd2b3e 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java @@ -39,11 +39,17 @@ public class TransferManager { private static final Logger logger = LogManager.getLogger(TransferManager.class); - private final BlobContainer blobContainer; + @FunctionalInterface + public interface BlobStreamReader { + InputStream read(String name, long position, long length) throws IOException; + } + + private final BlobStreamReader blobStreamReader; private final FileCache fileCache; - public TransferManager(final BlobContainer blobContainer, final FileCache fileCache) { - this.blobContainer = blobContainer; + + public TransferManager(final BlobStreamReader blobStreamReader, final FileCache fileCache) { + this.blobStreamReader = blobStreamReader; this.fileCache = fileCache; } @@ -59,7 +65,7 @@ public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOExceptio final CachedIndexInput cacheEntry = fileCache.compute(key, (path, cachedIndexInput) -> { if (cachedIndexInput == null || cachedIndexInput.isClosed()) { // Doesn't exist or is closed, either way create a new one - return new DelayedCreationCachedIndexInput(fileCache, blobContainer, blobFetchRequest); + return new DelayedCreationCachedIndexInput(fileCache, blobStreamReader, blobFetchRequest); } else { // already in the cache and ready to be used (open) return cachedIndexInput; @@ -77,7 +83,7 @@ public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOExceptio } @SuppressWarnings("removal") - private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobContainer blobContainer, BlobFetchRequest request) { + private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobStreamReader blobStreamReader, BlobFetchRequest request) { // We need to do a privileged action here in order to fetch from remote // and write to the local file cache in case this is invoked as a side // effect of a plugin (such as a scripted search) that doesn't have the @@ -91,7 +97,7 @@ private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobCo ) { for (BlobFetchRequest.BlobPart blobPart : request.blobParts()) { try ( - InputStream snapshotFileInputStream = blobContainer.readBlob( + InputStream snapshotFileInputStream = blobStreamReader.read( blobPart.getBlobName(), blobPart.getPosition(), blobPart.getLength() @@ -119,15 +125,15 @@ private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobCo */ private static class DelayedCreationCachedIndexInput implements CachedIndexInput { private final FileCache fileCache; - private final BlobContainer blobContainer; + private final BlobStreamReader blobStreamReader; private final BlobFetchRequest request; private final CompletableFuture result = new CompletableFuture<>(); private final AtomicBoolean isStarted = new AtomicBoolean(false); private final AtomicBoolean isClosed = new AtomicBoolean(false); - private DelayedCreationCachedIndexInput(FileCache fileCache, BlobContainer blobContainer, BlobFetchRequest request) { + private DelayedCreationCachedIndexInput(FileCache fileCache, BlobStreamReader blobStreamReader, BlobFetchRequest request) { this.fileCache = fileCache; - this.blobContainer = blobContainer; + this.blobStreamReader = blobStreamReader; this.request = request; } @@ -139,7 +145,7 @@ public IndexInput getIndexInput() throws IOException { if (isStarted.getAndSet(true) == false) { // We're the first one here, need to download the block try { - result.complete(createIndexInput(fileCache, blobContainer, request)); + result.complete(createIndexInput(fileCache, blobStreamReader, request)); } catch (Exception e) { result.completeExceptionally(e); fileCache.remove(request.getFilePath()); diff --git a/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTests.java b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTests.java index 7ae3944eb6944..c0a5ea749b765 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTests.java @@ -60,7 +60,7 @@ public void setUp() throws Exception { directory = new MMapDirectory(createTempDir(), SimpleFSLockFactory.INSTANCE); blobContainer = mock(BlobContainer.class); doAnswer(i -> new ByteArrayInputStream(createData())).when(blobContainer).readBlob(eq("blob"), anyLong(), anyLong()); - transferManager = new TransferManager(blobContainer, fileCache); + transferManager = new TransferManager(blobContainer::readBlob, fileCache); } @After From 3e7431877d0cd1c10ec3faad060935d95fd9956a Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Thu, 2 May 2024 22:41:48 +0530 Subject: [PATCH 08/22] Reuse OnDemandBlockSnapshotIndexInput instead of OnDemandBlockCompositeIndexInput Signed-off-by: Shreyansh Ray --- .../index/store/CompositeDirectory.java | 47 +++++++++++++++++-- .../file/OnDemandBlockSnapshotIndexInput.java | 8 +++- .../store/remote/utils/TransferManager.java | 4 ++ .../OnDemandBlockSnapshotIndexInputTests.java | 4 +- 4 files changed, 56 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 26faf9d2039c0..740aee3092e57 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -17,11 +17,15 @@ import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.util.Version; import org.opensearch.common.lucene.store.FilterIndexOutput; import org.opensearch.common.lucene.store.InputStreamIndexInput; +import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; import org.opensearch.index.store.remote.file.OnDemandCompositeBlockIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.filecache.WrappedCachedIndexInput; +import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.utils.BlobFetchRequest; import org.opensearch.index.store.remote.utils.BlockIOContext; import org.opensearch.index.store.remote.utils.FileType; @@ -29,6 +33,7 @@ import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InputStream; import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.util.ArrayList; @@ -54,6 +59,7 @@ public class CompositeDirectory extends FilterDirectory { private final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.WriteLock writeLock = readWriteLock.writeLock(); private final ReentrantReadWriteLock.ReadLock readLock = readWriteLock.readLock(); + private final RemoteDirectoryBlobStreamReader remoteDirectoryBlobStreamReader; /** * Constructor to initialise the composite directory @@ -66,8 +72,9 @@ public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirector this.localDirectory = localDirectory; this.remoteDirectory = remoteDirectory; this.fileCache = fileCache; + remoteDirectoryBlobStreamReader = new RemoteDirectoryBlobStreamReader(IOContext.DEFAULT, remoteDirectory); transferManager = new TransferManager( - (name, position, length) -> new InputStreamIndexInput(remoteDirectory.openInput(name, new BlockIOContext(IOContext.DEFAULT, position, length)), length), + remoteDirectoryBlobStreamReader, fileCache); } @@ -252,6 +259,7 @@ public IndexInput openInput(String name, IOContext context) throws IOException { logger.trace("openInput() called {}", name); writeLock.lock(); try { + remoteDirectoryBlobStreamReader.setContext(context); /** * We aren't tracking temporary files (created via createTempOutput) currently in FileCache as these are created and then deleted within a very short span of time * We will be reading them directory from the local directory @@ -277,7 +285,19 @@ else if (fileCache.get(localDirectory.getDirectory().resolve(name)) != null) { */ else { logger.trace("Complete file not in FileCache, to be fetched in Blocks from Remote"); - return new OnDemandCompositeBlockIndexInput(remoteDirectory, name, localDirectory, fileCache, context); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); + RemoteSegmentStoreDirectory.UploadedSegmentMetadata uploadedSegmentMetadata = remoteSegmentMetadata.getMetadata().get(name); + /** + * TODO : Refactor FileInfo and OnDemandBlockSnapshotIndexInput to more generic names as they are not Remote Snapshot specific + */ + BlobStoreIndexShardSnapshot.FileInfo fileInfo = new BlobStoreIndexShardSnapshot.FileInfo( + name, + new StoreFileMetadata(name, uploadedSegmentMetadata.getLength(), + uploadedSegmentMetadata.getChecksum(), Version.LATEST), + null + ); + return new OnDemandBlockSnapshotIndexInput(fileInfo, localDirectory, transferManager); + //return new OnDemandCompositeBlockIndexInput(remoteDirectory, name, localDirectory, fileCache, context); } } finally { writeLock.unlock(); @@ -350,7 +370,7 @@ public void afterSyncToRemote(Collection files) throws IOException { * Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality */ logger.trace("File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", fileName); - // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); + fileCache.remove(localDirectory.getDirectory().resolve(fileName)); } finally { writeLock.unlock(); } @@ -369,7 +389,7 @@ private String[] getRemoteFiles() throws IOException { try { remoteFiles = remoteDirectory.listAll(); } catch (NullPointerException e) { - /** + /* * There are two scenarios where the listAll() call on remote directory returns NullPointerException: * - When remote directory is not set * - When init() of remote directory has not yet been called @@ -401,4 +421,23 @@ public void close() throws IOException { cacheFile(fileName); } } + + private class RemoteDirectoryBlobStreamReader implements TransferManager.BlobStreamReader { + private IOContext context; + private final RemoteSegmentStoreDirectory remoteDirectory; + + RemoteDirectoryBlobStreamReader(IOContext context, RemoteSegmentStoreDirectory remoteDirectory) { + this.context = context; + this.remoteDirectory = remoteDirectory; + } + + void setContext(IOContext context) { + this.context = context; + } + + @Override + public InputStream read(String name, long position, long length) throws IOException { + return new InputStreamIndexInput(remoteDirectory.openInput(name, new BlockIOContext(context, position, length)), length); + } + } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java index 8097fd08da50a..1f70fd132ce17 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java @@ -8,9 +8,12 @@ package org.opensearch.index.store.remote.file; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.IndexInput; import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; +import org.opensearch.index.store.CompositeDirectory; import org.opensearch.index.store.remote.utils.BlobFetchRequest; import org.opensearch.index.store.remote.utils.TransferManager; @@ -26,6 +29,7 @@ * @opensearch.internal */ public class OnDemandBlockSnapshotIndexInput extends OnDemandBlockIndexInput { + private static final Logger logger = LogManager.getLogger(OnDemandBlockSnapshotIndexInput.class); /** * Where this class fetches IndexInput parts from */ @@ -133,10 +137,12 @@ protected OnDemandBlockSnapshotIndexInput buildSlice(String sliceDescription, lo @Override protected IndexInput fetchBlock(int blockId) throws IOException { - final String blockFileName = fileName + "." + blockId; + logger.trace("fetchBlock called with blockId -> {}", blockId); + final String blockFileName = fileName + "_block_" + blockId; final long blockStart = getBlockStart(blockId); final long blockEnd = blockStart + getActualBlockSize(blockId); + logger.trace("File: {} , Block File: {} , BlockStart: {} , BlockEnd: {} , OriginalFileSize: {}", fileName, blockFileName, blockStart, blockEnd, originalFileSize); // Block may be present on multiple chunks of a file, so we need // to fetch each chunk/blob part separately to fetch an entire block. diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java index 557a71abd2b3e..f123a923f5ef6 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java @@ -61,12 +61,15 @@ public TransferManager(final BlobStreamReader blobStreamReader, final FileCache public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOException { final Path key = blobFetchRequest.getFilePath(); + logger.trace("fetchBlob called for {}", key.toString()); final CachedIndexInput cacheEntry = fileCache.compute(key, (path, cachedIndexInput) -> { if (cachedIndexInput == null || cachedIndexInput.isClosed()) { + logger.trace("Transfer Manager - IndexInput closed or not in cache"); // Doesn't exist or is closed, either way create a new one return new DelayedCreationCachedIndexInput(fileCache, blobStreamReader, blobFetchRequest); } else { + logger.trace("Transfer Manager - Already in cache"); // already in the cache and ready to be used (open) return cachedIndexInput; } @@ -91,6 +94,7 @@ private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobSt return AccessController.doPrivileged((PrivilegedAction) () -> { try { if (Files.exists(request.getFilePath()) == false) { + logger.trace("Fetching from Remote in createIndexInput of Transfer Manager"); try ( OutputStream fileOutputStream = Files.newOutputStream(request.getFilePath()); OutputStream localFileOutputStream = new BufferedOutputStream(fileOutputStream) diff --git a/server/src/test/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInputTests.java b/server/src/test/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInputTests.java index a135802c5f49c..c7d0cc0c5b96e 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInputTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInputTests.java @@ -207,7 +207,7 @@ private void initBlockFiles(int blockSize, FSDirectory fsDirectory) { // write 48, -80 alternatively for (int i = 0; i < numOfBlocks; i++) { // create normal blocks - String blockName = BLOCK_FILE_PREFIX + "." + i; + String blockName = BLOCK_FILE_PREFIX + "_block_" + i; IndexOutput output = fsDirectory.createOutput(blockName, null); // since block size is always even number, safe to do division for (int j = 0; j < blockSize / 2; j++) { @@ -221,7 +221,7 @@ private void initBlockFiles(int blockSize, FSDirectory fsDirectory) { if (numOfBlocks > 1 && sizeOfLastBlock != 0) { // create last block - String lastBlockName = BLOCK_FILE_PREFIX + "." + numOfBlocks; + String lastBlockName = BLOCK_FILE_PREFIX + "_block_" + numOfBlocks; IndexOutput output = fsDirectory.createOutput(lastBlockName, null); for (int i = 0; i < sizeOfLastBlock; i++) { if ((i & 1) == 0) { From dca1a0776cc4c0afec6b941eb56729dafa620ff7 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Wed, 8 May 2024 23:34:36 +0530 Subject: [PATCH 09/22] Modify constructors to avoid breaking public api contract and code review fixes Signed-off-by: Shreyansh Ray --- .../remotestore/CompositeDirectoryIT.java | 58 +++--- .../metadata/MetadataCreateIndexService.java | 4 +- .../org/opensearch/index/IndexModule.java | 23 +++ .../org/opensearch/index/IndexService.java | 78 +++++++- .../shard/RemoteStoreRefreshListener.java | 16 +- .../store/CloseableFilterIndexOutput.java | 45 +++++ .../index/store/CompositeDirectory.java | 170 ++++++---------- .../file/OnDemandBlockSnapshotIndexInput.java | 10 +- .../OnDemandCompositeBlockIndexInput.java | 137 ------------- ...put.java => NonBlockCachedIndexInput.java} | 12 +- .../store/remote/utils/TransferManager.java | 27 +-- .../opensearch/indices/IndicesService.java | 63 +++++- .../opensearch/index/IndexModuleTests.java | 12 +- .../BaseRemoteSegmentStoreDirectoryTests.java | 178 +++++++++++++++++ .../index/store/CompositeDirectoryTests.java | 189 ++++++++++++++++++ .../RemoteSegmentStoreDirectoryTests.java | 153 +------------- .../snapshots/SnapshotResiliencyTests.java | 3 +- .../test/OpenSearchIntegTestCase.java | 6 +- 18 files changed, 723 insertions(+), 461 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java delete mode 100644 server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java rename server/src/main/java/org/opensearch/index/store/remote/filecache/{WrappedCachedIndexInput.java => NonBlockCachedIndexInput.java} (77%) create mode 100644 server/src/test/java/org/opensearch/index/store/BaseRemoteSegmentStoreDirectoryTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java index 71255236a3809..a34b665561289 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java @@ -14,30 +14,40 @@ import org.apache.lucene.store.FilterDirectory; import org.opensearch.action.admin.indices.get.GetIndexRequest; import org.opensearch.action.admin.indices.get.GetIndexResponse; +import org.opensearch.action.search.SearchResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexService; +import org.opensearch.index.query.QueryBuilders; import org.opensearch.index.shard.IndexShard; import org.opensearch.index.store.CompositeDirectory; import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; import org.opensearch.indices.IndicesService; -import org.opensearch.indices.replication.common.ReplicationType; import org.opensearch.test.OpenSearchIntegTestCase; import org.opensearch.test.junit.annotations.TestLogging; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_SEGMENT_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_STORE_ENABLED; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY; -import static org.opensearch.cluster.metadata.IndexMetadata.SETTING_REPLICATION_TYPE; +import java.util.Map; + import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) -@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST) -//@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) +// Uncomment the below line to enable trace level logs for this test for better debugging +@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") public class CompositeDirectoryIT extends RemoteStoreBaseIntegTestCase { + /* + Disabling MockFSIndexStore plugin as the MockFSDirectoryFactory wraps the FSDirectory over a OpenSearchMockDirectoryWrapper which extends FilterDirectory (whereas FSDirectory extends BaseDirectory) + As a result of this wrapping the local directory of Composite Directory does not satisfy the assertion that local directory must be of type FSDirectory + */ + @Override + protected boolean addMockIndexStorePlugin() { + return false; + } + @Override protected Settings featureFlagSettings() { Settings.Builder featureSettings = Settings.builder(); @@ -53,32 +63,28 @@ public void testCompositeDirectory() throws Exception { .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), "partial") .build(); assertAcked(client().admin().indices().prepareCreate("test-idx-1").setSettings(settings).get()); + + // Check if the Directory initialized for the IndexShard is of Composite Directory type + IndexService indexService = internalCluster().getDataNodeInstance(IndicesService.class).indexService(resolveIndex("test-idx-1")); + IndexShard shard = indexService.getShardOrNull(0); + Directory directory = (((FilterDirectory) (((FilterDirectory) (shard.store().directory())).getDelegate())).getDelegate()); + assertTrue(directory instanceof CompositeDirectory); + + // Verify from the cluster settings if the data locality is partial GetIndexResponse getIndexResponse = client().admin() .indices() .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) .get(); - boolean indexServiceFound = false; - String[] nodes = internalCluster().getNodeNames(); - for (String node : nodes) { - IndexService indexService = internalCluster().getInstance(IndicesService.class, node).indexService(resolveIndex("test-idx-1")); - if (indexService == null) { - continue; - } - IndexShard shard = indexService.getShardOrNull(0); - Directory directory = (((FilterDirectory) (((FilterDirectory) (shard.store().directory())).getDelegate())).getDelegate()); - assertTrue(directory instanceof CompositeDirectory); - indexServiceFound = true; - } - assertTrue(indexServiceFound); Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - assertEquals(ReplicationType.SEGMENT.toString(), indexSettings.get(SETTING_REPLICATION_TYPE)); - assertEquals("true", indexSettings.get(SETTING_REMOTE_STORE_ENABLED)); - assertEquals(REPOSITORY_NAME, indexSettings.get(SETTING_REMOTE_SEGMENT_STORE_REPOSITORY)); - assertEquals(REPOSITORY_2_NAME, indexSettings.get(SETTING_REMOTE_TRANSLOG_STORE_REPOSITORY)); assertEquals("partial", indexSettings.get("index.store.data_locality")); + // Index data and ensure cluster does not turn red while indexing + Map stats = indexData(10, false, "test-idx-1"); + refresh("test-idx-1"); ensureGreen("test-idx-1"); - indexData(10, false, "test-idx-1"); - ensureGreen("test-idx-1"); + + // Search and verify that the total docs indexed match the search hits + SearchResponse searchResponse3 = client().prepareSearch("test-idx-1").setQuery(QueryBuilders.matchAllQuery()).get(); + assertHitCount(searchResponse3, stats.get(TOTAL_OPERATIONS)); } } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 8999d3c348c51..014bd6b1885ea 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1696,7 +1696,9 @@ public static void validateIndexStoreLocality(Settings indexSettings) { .equalsIgnoreCase(IndexModule.DataLocalityType.PARTIAL.toString()) && !FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING)) { throw new IllegalArgumentException( - "index.store.locality can be set to PARTIAL only if Feature Flag for Writable Remote Index is true" + "index.store.locality can be set to PARTIAL only if Feature Flag [" + + FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING.getKey() + + "] is set to true" ); } } diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index b4fd7666ba45a..d79f19f17e167 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -345,6 +345,29 @@ public IndexModule( this.fileCache = fileCache; } + public IndexModule( + final IndexSettings indexSettings, + final AnalysisRegistry analysisRegistry, + final EngineFactory engineFactory, + final EngineConfigFactory engineConfigFactory, + final Map directoryFactories, + final BooleanSupplier allowExpensiveQueries, + final IndexNameExpressionResolver expressionResolver, + final Map recoveryStateFactories + ) { + this( + indexSettings, + analysisRegistry, + engineFactory, + engineConfigFactory, + directoryFactories, + allowExpensiveQueries, + expressionResolver, + recoveryStateFactories, + null + ); + } + /** * Adds a Setting and it's consumer for this index. */ diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 08a9779a23c02..78bfedb7b6d65 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -312,6 +312,81 @@ public IndexService( updateFsyncTaskIfNecessary(); } + public IndexService( + IndexSettings indexSettings, + IndexCreationContext indexCreationContext, + NodeEnvironment nodeEnv, + NamedXContentRegistry xContentRegistry, + SimilarityService similarityService, + ShardStoreDeleter shardStoreDeleter, + IndexAnalyzers indexAnalyzers, + EngineFactory engineFactory, + EngineConfigFactory engineConfigFactory, + CircuitBreakerService circuitBreakerService, + BigArrays bigArrays, + ThreadPool threadPool, + ScriptService scriptService, + ClusterService clusterService, + Client client, + QueryCache queryCache, + IndexStorePlugin.DirectoryFactory directoryFactory, + IndexStorePlugin.DirectoryFactory remoteDirectoryFactory, + IndexEventListener eventListener, + Function> wrapperFactory, + MapperRegistry mapperRegistry, + IndicesFieldDataCache indicesFieldDataCache, + List searchOperationListeners, + List indexingOperationListeners, + NamedWriteableRegistry namedWriteableRegistry, + BooleanSupplier idFieldDataEnabled, + BooleanSupplier allowExpensiveQueries, + IndexNameExpressionResolver expressionResolver, + ValuesSourceRegistry valuesSourceRegistry, + IndexStorePlugin.RecoveryStateFactory recoveryStateFactory, + BiFunction translogFactorySupplier, + Supplier clusterDefaultRefreshIntervalSupplier, + RecoverySettings recoverySettings, + RemoteStoreSettings remoteStoreSettings + ) { + this( + indexSettings, + indexCreationContext, + nodeEnv, + xContentRegistry, + similarityService, + shardStoreDeleter, + indexAnalyzers, + engineFactory, + engineConfigFactory, + circuitBreakerService, + bigArrays, + threadPool, + scriptService, + clusterService, + client, + queryCache, + directoryFactory, + remoteDirectoryFactory, + eventListener, + wrapperFactory, + mapperRegistry, + indicesFieldDataCache, + searchOperationListeners, + indexingOperationListeners, + namedWriteableRegistry, + idFieldDataEnabled, + allowExpensiveQueries, + expressionResolver, + valuesSourceRegistry, + recoveryStateFactory, + translogFactorySupplier, + clusterDefaultRefreshIntervalSupplier, + recoverySettings, + remoteStoreSettings, + null + ); + } + static boolean needsMapperService(IndexSettings indexSettings, IndexCreationContext indexCreationContext) { return false == (indexSettings.getIndexMetadata().getState() == IndexMetadata.State.CLOSE && indexCreationContext == IndexCreationContext.CREATE_INDEX); // metadata verification needs a mapper service @@ -542,7 +617,7 @@ public synchronized IndexShard createShard( if (FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING) && // TODO : Need to remove this check after support for hot indices is added in Composite Directory this.indexSettings.isStoreLocalityPartial()) { - /** + /* * Currently Composite Directory only supports local directory to be of type FSDirectory * The reason is that FileCache currently has it key type as Path * Composite Directory currently uses FSDirectory's getDirectory() method to fetch and use the Path for operating on FileCache @@ -550,6 +625,7 @@ public synchronized IndexShard createShard( */ Directory localDirectory = directoryFactory.newDirectory(this.indexSettings, path); assert localDirectory instanceof FSDirectory : "For Composite Directory, local directory must be of type FSDirectory"; + assert fileCache != null : "File Cache not initialized on this Node, cannot create Composite Directory without FileCache"; directory = new CompositeDirectory((FSDirectory) localDirectory, (RemoteSegmentStoreDirectory) remoteDirectory, fileCache); } else { directory = directoryFactory.newDirectory(this.indexSettings, path); 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 9101d4e066794..e9f996e24f679 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -262,6 +262,12 @@ private boolean syncSegments() { Map localSegmentsSizeMap = updateLocalSizeMapAndTracker(localSegmentsPostRefresh).entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + Collection segmentsToRefresh = localSegmentsPostRefresh.stream() + .filter(file -> !skipUpload(file)) + .collect(Collectors.toList()); + Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); + CountDownLatch latch = new CountDownLatch(1); ActionListener segmentUploadsCompletedListener = new LatchedActionListener<>(new ActionListener<>() { @Override @@ -283,6 +289,9 @@ public void onResponse(Void unused) { // At this point since we have uploaded new segments, segment infos and segment metadata file, // along with marking minSeqNoToKeep, upload has succeeded completely. successful.set(true); + if (directory instanceof CompositeDirectory) { + ((CompositeDirectory) directory).afterSyncToRemote(segmentsToRefresh); + } } catch (Exception e) { // We don't want to fail refresh if upload of new segments fails. The missed segments will be re-tried // as part of exponential back-off retry logic. This should not affect durability of the indexed data @@ -297,15 +306,8 @@ public void onFailure(Exception e) { } }, latch); - Collection segmentsToRefresh = localSegmentsPostRefresh.stream() - .filter(file -> !skipUpload(file)) - .collect(Collectors.toList()); // Start the segments files upload uploadNewSegments(localSegmentsPostRefresh, localSegmentsSizeMap, segmentUploadsCompletedListener); - Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); - if (directory instanceof CompositeDirectory) { - ((CompositeDirectory) directory).afterSyncToRemote(segmentsToRefresh); - } if (latch.await( remoteStoreSettings.getClusterRemoteSegmentTransferTimeout().millis(), TimeUnit.MILLISECONDS diff --git a/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java new file mode 100644 index 0000000000000..8df5d648b91f3 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java @@ -0,0 +1,45 @@ +/* + * 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.store; + +import org.apache.lucene.store.IndexOutput; +import org.opensearch.common.lucene.store.FilterIndexOutput; + +import java.io.IOException; + +/** + * FilterIndexOutput which takes in an additional FunctionalInterface as a parameter to perform required operations once the IndexOutput is closed + * + * @opensearch.internal + */ +public class CloseableFilterIndexOutput extends FilterIndexOutput { + + /** + * Functional Interface which takes the name of the file as input on which the required operations are to be performed + */ + @FunctionalInterface + public interface OnCloseListener { + void onClose(String name); + } + + OnCloseListener onCloseListener; + String fileName; + + public CloseableFilterIndexOutput(IndexOutput out, String fileName, OnCloseListener onCloseListener) { + super("CloseableFilterIndexOutput for file " + fileName, out); + this.fileName = fileName; + this.onCloseListener = onCloseListener; + } + + @Override + public void close() throws IOException { + super.close(); + onCloseListener.onClose(fileName); + } +} diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 740aee3092e57..5536be1212ccb 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -15,28 +15,23 @@ import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.Lock; -import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.Version; -import org.opensearch.common.lucene.store.FilterIndexOutput; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; -import org.opensearch.index.store.remote.file.OnDemandCompositeBlockIndexInput; +import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.filecache.WrappedCachedIndexInput; +import org.opensearch.index.store.remote.filecache.NonBlockCachedIndexInput; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; -import org.opensearch.index.store.remote.utils.BlobFetchRequest; import org.opensearch.index.store.remote.utils.BlockIOContext; import org.opensearch.index.store.remote.utils.FileType; import org.opensearch.index.store.remote.utils.TransferManager; import java.io.FileNotFoundException; import java.io.IOException; -import java.io.InputStream; import java.nio.file.NoSuchFileException; import java.nio.file.Path; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; @@ -49,7 +44,10 @@ * Consumers of Composite directory need not worry whether file is in local or remote * All such abstractions will be handled by the Composite directory itself * Implements all required methods by Directory abstraction + * + * @opensearch.internal */ +@ExperimentalApi public class CompositeDirectory extends FilterDirectory { private static final Logger logger = LogManager.getLogger(CompositeDirectory.class); private final FSDirectory localDirectory; @@ -59,7 +57,6 @@ public class CompositeDirectory extends FilterDirectory { private final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.WriteLock writeLock = readWriteLock.writeLock(); private final ReentrantReadWriteLock.ReadLock readLock = readWriteLock.readLock(); - private final RemoteDirectoryBlobStreamReader remoteDirectoryBlobStreamReader; /** * Constructor to initialise the composite directory @@ -72,10 +69,13 @@ public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirector this.localDirectory = localDirectory; this.remoteDirectory = remoteDirectory; this.fileCache = fileCache; - remoteDirectoryBlobStreamReader = new RemoteDirectoryBlobStreamReader(IOContext.DEFAULT, remoteDirectory); transferManager = new TransferManager( - remoteDirectoryBlobStreamReader, - fileCache); + (name, position, length) -> new InputStreamIndexInput( + remoteDirectory.openInput(name, new BlockIOContext(IOContext.DEFAULT, position, length)), + length + ), + fileCache + ); } /** @@ -86,17 +86,15 @@ public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirector */ @Override public String[] listAll() throws IOException { - logger.trace("listAll() called ..."); + logger.trace("listAll() called"); readLock.lock(); try { String[] localFiles = localDirectory.listAll(); - logger.trace("LocalDirectory files : {}", () -> Arrays.toString(localFiles)); + logger.trace("Local Directory files : {}", () -> Arrays.toString(localFiles)); Set allFiles = new HashSet<>(Arrays.asList(localFiles)); - if (remoteDirectory != null) { - String[] remoteFiles = getRemoteFiles(); - allFiles.addAll(Arrays.asList(remoteFiles)); - logger.trace("Remote Directory files : {}", () -> Arrays.toString(remoteFiles)); - } + String[] remoteFiles = getRemoteFiles(); + allFiles.addAll(Arrays.asList(remoteFiles)); + logger.trace("Remote Directory files : {}", () -> Arrays.toString(remoteFiles)); Set localLuceneFiles = allFiles.stream() .filter(file -> !FileType.isBlockFile(file)) .collect(Collectors.toUnmodifiableSet()); @@ -112,7 +110,7 @@ public String[] listAll() throws IOException { /** * Removes an existing file in the directory. - * Throws {@link NoSuchFileException} or {@link FileNotFoundException} in case file is not present locally and in remote as well + * Currently deleting only from local directory as files from remote should not be deleted due to availability reasons * @param name the name of an existing file. * @throws IOException in case of I/O error */ @@ -121,11 +119,11 @@ public void deleteFile(String name) throws IOException { logger.trace("deleteFile() called {}", name); writeLock.lock(); try { - localDirectory.deleteFile(name); + /* + Not deleting from localDirectory directly since it causes a race condition when the localDirectory deletes a file, and it ends up in pendingDeletion state. + Meanwhile, fileCache on removal deletes the file directly via the Files class and later when the directory tries to delete the files pending for deletion (which happens before creating a new file), it causes NoSuchFileException and new file creation fails + */ fileCache.remove(localDirectory.getDirectory().resolve(name)); - } catch (NoSuchFileException | FileNotFoundException e) { - logger.trace("File {} not found in local, trying to delete from Remote", name); - remoteDirectory.deleteFile(name); } finally { writeLock.unlock(); } @@ -143,8 +141,10 @@ public long fileLength(String name) throws IOException { readLock.lock(); try { long fileLength; - if (isTempFile(name) || fileCache.get(localDirectory.getDirectory().resolve(name)) != null) { + Path key = localDirectory.getDirectory().resolve(name); + if (isTempFile(name) || fileCache.get(key) != null) { fileLength = localDirectory.fileLength(name); + fileCache.decRef(key); logger.trace("fileLength from Local {}", fileLength); } else { fileLength = remoteDirectory.fileLength(name); @@ -167,10 +167,16 @@ public IndexOutput createOutput(String name, IOContext context) throws IOExcepti logger.trace("createOutput() called {}", name); writeLock.lock(); try { - /** - * The CacheableIndexOutput will ensure that the file is added to FileCache once write is completed on this file + /* + * The CloseableFilterIndexOutput will ensure that the file is added to FileCache once write is completed on this file */ - return new CacheableIndexOutput(localDirectory.createOutput(name, context), name); + return new CloseableFilterIndexOutput(localDirectory.createOutput(name, context), name, (fileName) -> { + try { + cacheFile(fileName); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); } finally { writeLock.unlock(); } @@ -203,13 +209,12 @@ public void sync(Collection names) throws IOException { logger.trace("sync() called {}", names); writeLock.lock(); try { - Collection newLocalFiles = new ArrayList<>(); Collection remoteFiles = Arrays.asList(getRemoteFiles()); - for (String name : names) { - if (remoteFiles.contains(name) == false) newLocalFiles.add(name); - } - logger.trace("Synced files : {}", newLocalFiles); - localDirectory.sync(newLocalFiles); + Collection filesToSync = names.stream() + .filter(name -> remoteFiles.contains(name) == false) + .collect(Collectors.toList()); + logger.trace("Synced files : {}", filesToSync); + localDirectory.sync(filesToSync); } finally { writeLock.unlock(); } @@ -259,69 +264,49 @@ public IndexInput openInput(String name, IOContext context) throws IOException { logger.trace("openInput() called {}", name); writeLock.lock(); try { - remoteDirectoryBlobStreamReader.setContext(context); - /** + /* * We aren't tracking temporary files (created via createTempOutput) currently in FileCache as these are created and then deleted within a very short span of time * We will be reading them directory from the local directory */ if (isTempFile(name)) { return localDirectory.openInput(name, context); } - /** + /* * Return directly from the FileCache (via TransferManager) if complete file is present */ - else if (fileCache.get(localDirectory.getDirectory().resolve(name)) != null) { + + Path key = localDirectory.getDirectory().resolve(name); + CachedIndexInput indexInput = fileCache.get(key); + if (indexInput != null) { logger.trace("Complete file found in FileCache"); - BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() - .directory(localDirectory) - .fileName(name) - // position and length are not required here since this is a complete file, just adding dummy values for validation - .blobParts(new ArrayList<>(Arrays.asList(new BlobFetchRequest.BlobPart(name, 0, 1)))) - .build(); - return transferManager.fetchBlob(blobFetchRequest); + try { + return indexInput.getIndexInput().clone(); + } finally { + fileCache.decRef(key); + } } - /** + /* * If file has been uploaded to the Remote Store, fetch it from the Remote Store in blocks via OnDemandCompositeBlockIndexInput */ else { logger.trace("Complete file not in FileCache, to be fetched in Blocks from Remote"); RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); RemoteSegmentStoreDirectory.UploadedSegmentMetadata uploadedSegmentMetadata = remoteSegmentMetadata.getMetadata().get(name); - /** + /* * TODO : Refactor FileInfo and OnDemandBlockSnapshotIndexInput to more generic names as they are not Remote Snapshot specific */ BlobStoreIndexShardSnapshot.FileInfo fileInfo = new BlobStoreIndexShardSnapshot.FileInfo( name, - new StoreFileMetadata(name, uploadedSegmentMetadata.getLength(), - uploadedSegmentMetadata.getChecksum(), Version.LATEST), + new StoreFileMetadata(name, uploadedSegmentMetadata.getLength(), uploadedSegmentMetadata.getChecksum(), Version.LATEST), null ); return new OnDemandBlockSnapshotIndexInput(fileInfo, localDirectory, transferManager); - //return new OnDemandCompositeBlockIndexInput(remoteDirectory, name, localDirectory, fileCache, context); } } finally { writeLock.unlock(); } } - /** - * Acquires and returns a {@link Lock} for a file with the given name. - * @param name the name of the lock file - * @throws LockObtainFailedException (optional specific exception) if the lock could not be - * obtained because it is currently held elsewhere. - * @throws IOException in case of I/O error - */ - @Override - public Lock obtainLock(String name) throws IOException { - logger.trace("obtainLock() called {}", name); - writeLock.lock(); - try { - return localDirectory.obtainLock(name); - } finally { - writeLock.unlock(); - } - } - /** * Closes the directory * @throws IOException in case of I/O error @@ -330,7 +315,12 @@ public Lock obtainLock(String name) throws IOException { public void close() throws IOException { writeLock.lock(); try { + Arrays.stream(localDirectory.listAll()).forEach(f -> { + logger.trace("Removing file from cache {}", f); + fileCache.remove(localDirectory.getDirectory().resolve(f)); + }); localDirectory.close(); + remoteDirectory.close(); } finally { writeLock.unlock(); } @@ -366,11 +356,11 @@ public void afterSyncToRemote(Collection files) throws IOException { writeLock.lock(); try { /** - * TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache + * TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache * Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality */ logger.trace("File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", fileName); - fileCache.remove(localDirectory.getDirectory().resolve(fileName)); + // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); } finally { writeLock.unlock(); } @@ -403,41 +393,11 @@ private String[] getRemoteFiles() throws IOException { private void cacheFile(String name) throws IOException { Path filePath = localDirectory.getDirectory().resolve(name); - fileCache.put(filePath, new WrappedCachedIndexInput(localDirectory.openInput(name, IOContext.READ))); - } - - private class CacheableIndexOutput extends FilterIndexOutput { - - String fileName; - - public CacheableIndexOutput(IndexOutput out, String fileName) { - super("CacheableIndexOutput for file : " + fileName, out); - this.fileName = fileName; - } - - @Override - public void close() throws IOException { - super.close(); - cacheFile(fileName); - } + fileCache.put(filePath, new NonBlockCachedIndexInput(localDirectory.openInput(name, IOContext.READ))); + // Decrementing ref here as above put call increments the ref of the key + fileCache.decRef(filePath); + // TODO : Pin the above filePath in the file cache once pinning support is added so that it cannot be evicted unless it has been + // successfully uploaded to Remote } - private class RemoteDirectoryBlobStreamReader implements TransferManager.BlobStreamReader { - private IOContext context; - private final RemoteSegmentStoreDirectory remoteDirectory; - - RemoteDirectoryBlobStreamReader(IOContext context, RemoteSegmentStoreDirectory remoteDirectory) { - this.context = context; - this.remoteDirectory = remoteDirectory; - } - - void setContext(IOContext context) { - this.context = context; - } - - @Override - public InputStream read(String name, long position, long length) throws IOException { - return new InputStreamIndexInput(remoteDirectory.openInput(name, new BlockIOContext(context, position, length)), length); - } - } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java index 1f70fd132ce17..ad56127394779 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandBlockSnapshotIndexInput.java @@ -13,7 +13,6 @@ import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.IndexInput; import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo; -import org.opensearch.index.store.CompositeDirectory; import org.opensearch.index.store.remote.utils.BlobFetchRequest; import org.opensearch.index.store.remote.utils.TransferManager; @@ -142,7 +141,14 @@ protected IndexInput fetchBlock(int blockId) throws IOException { final long blockStart = getBlockStart(blockId); final long blockEnd = blockStart + getActualBlockSize(blockId); - logger.trace("File: {} , Block File: {} , BlockStart: {} , BlockEnd: {} , OriginalFileSize: {}", fileName, blockFileName, blockStart, blockEnd, originalFileSize); + logger.trace( + "File: {} , Block File: {} , BlockStart: {} , BlockEnd: {} , OriginalFileSize: {}", + fileName, + blockFileName, + blockStart, + blockEnd, + originalFileSize + ); // Block may be present on multiple chunks of a file, so we need // to fetch each chunk/blob part separately to fetch an entire block. diff --git a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java deleted file mode 100644 index f024592330512..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/remote/file/OnDemandCompositeBlockIndexInput.java +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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.store.remote.file; - -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import org.apache.lucene.store.FSDirectory; -import org.apache.lucene.store.IOContext; -import org.apache.lucene.store.IndexInput; -import org.opensearch.common.lucene.store.InputStreamIndexInput; -import org.opensearch.index.store.RemoteSegmentStoreDirectory; -import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.utils.BlobFetchRequest; -import org.opensearch.index.store.remote.utils.BlockIOContext; -import org.opensearch.index.store.remote.utils.TransferManager; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * OnDemandCompositeBlockIndexInput is used by the Composite Directory to read data in blocks from Remote and cache those blocks in FileCache - */ -public class OnDemandCompositeBlockIndexInput extends OnDemandBlockIndexInput { - - private static final Logger logger = LogManager.getLogger(OnDemandCompositeBlockIndexInput.class); - private final RemoteSegmentStoreDirectory remoteDirectory; - private final String fileName; - private final Long originalFileSize; - private final FSDirectory localDirectory; - private final IOContext context; - private final FileCache fileCache; - private final TransferManager transferManager; - - public OnDemandCompositeBlockIndexInput( - RemoteSegmentStoreDirectory remoteDirectory, - String fileName, - FSDirectory localDirectory, - FileCache fileCache, - IOContext context - ) throws IOException { - this( - OnDemandBlockIndexInput.builder() - .resourceDescription("OnDemandCompositeBlockIndexInput") - .isClone(false) - .offset(0L) - .length(remoteDirectory.fileLength(fileName)), - remoteDirectory, - fileName, - localDirectory, - fileCache, - context - ); - } - - public OnDemandCompositeBlockIndexInput( - Builder builder, - RemoteSegmentStoreDirectory remoteDirectory, - String fileName, - FSDirectory localDirectory, - FileCache fileCache, - IOContext context - ) throws IOException { - super(builder); - this.remoteDirectory = remoteDirectory; - this.localDirectory = localDirectory; - this.fileName = fileName; - this.fileCache = fileCache; - this.context = context; - this.transferManager = new TransferManager( - (name, position, length) -> new InputStreamIndexInput(remoteDirectory.openInput(name, new BlockIOContext(context, position, length)), length), - fileCache); - originalFileSize = remoteDirectory.fileLength(fileName); - } - - @Override - protected OnDemandCompositeBlockIndexInput buildSlice(String sliceDescription, long offset, long length) { - try { - return new OnDemandCompositeBlockIndexInput( - OnDemandBlockIndexInput.builder() - .blockSizeShift(blockSizeShift) - .isClone(true) - .offset(this.offset + offset) - .length(length) - .resourceDescription(sliceDescription), - remoteDirectory, - fileName, - localDirectory, - fileCache, - context - ); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - protected IndexInput fetchBlock(int blockId) throws IOException { - logger.trace("fetchBlock called with blockId -> {}", blockId); - final String blockFileName = fileName + "_block_" + blockId; - final long blockStart = getBlockStart(blockId); - final long length = getActualBlockSize(blockId); - logger.trace( - "File: {} , Block File: {} , Length: {} , BlockSize: {} , OriginalFileSize: {}", - fileName, - blockFileName, - blockStart, - length, - originalFileSize - ); - BlobFetchRequest blobFetchRequest = BlobFetchRequest.builder() - .directory(localDirectory) - .fileName(blockFileName) - .blobParts(new ArrayList<>(List.of(new BlobFetchRequest.BlobPart(fileName, blockStart, length)))) - .build(); - return transferManager.fetchBlob(blobFetchRequest); - } - - @Override - public OnDemandBlockIndexInput clone() { - OnDemandCompositeBlockIndexInput clone = buildSlice("clone", 0L, this.length); - // ensures that clones may be positioned at the same point as the blocked file they were cloned from - clone.cloneBlock(this); - return clone; - } - - private long getActualBlockSize(int blockId) { - return (blockId != getBlock(originalFileSize - 1)) ? blockSize : getBlockOffset(originalFileSize - 1) + 1; - } - -} diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/WrappedCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/NonBlockCachedIndexInput.java similarity index 77% rename from server/src/main/java/org/opensearch/index/store/remote/filecache/WrappedCachedIndexInput.java rename to server/src/main/java/org/opensearch/index/store/remote/filecache/NonBlockCachedIndexInput.java index 67b12a10810ed..ee520dc22964e 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/WrappedCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/NonBlockCachedIndexInput.java @@ -14,17 +14,17 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * Wrapper implementation of the CachedIndexInput which takes in an IndexInput as parameter + * Implementation of the CachedIndexInput for NON_BLOCK files which takes in an IndexInput as parameter */ -public class WrappedCachedIndexInput implements CachedIndexInput { +public class NonBlockCachedIndexInput implements CachedIndexInput { - IndexInput indexInput; - AtomicBoolean isClosed; + private final IndexInput indexInput; + private final AtomicBoolean isClosed; /** * Constructor - takes IndexInput as parameter */ - public WrappedCachedIndexInput(IndexInput indexInput) { + public NonBlockCachedIndexInput(IndexInput indexInput) { this.indexInput = indexInput; isClosed = new AtomicBoolean(false); } @@ -58,7 +58,7 @@ public boolean isClosed() { */ @Override public void close() throws Exception { - if (!isClosed()) { + if (!isClosed.getAndSet(true)) { indexInput.close(); isClosed.set(true); } diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java index f123a923f5ef6..df26f2f0925f6 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/TransferManager.java @@ -12,7 +12,6 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.filecache.FileCachedIndexInput; @@ -39,17 +38,19 @@ public class TransferManager { private static final Logger logger = LogManager.getLogger(TransferManager.class); + /** + * Functional interface to get an InputStream for a file at a certain offset and size + */ @FunctionalInterface - public interface BlobStreamReader { + public interface StreamReader { InputStream read(String name, long position, long length) throws IOException; } - private final BlobStreamReader blobStreamReader; + private final StreamReader streamReader; private final FileCache fileCache; - - public TransferManager(final BlobStreamReader blobStreamReader, final FileCache fileCache) { - this.blobStreamReader = blobStreamReader; + public TransferManager(final StreamReader streamReader, final FileCache fileCache) { + this.streamReader = streamReader; this.fileCache = fileCache; } @@ -67,7 +68,7 @@ public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOExceptio if (cachedIndexInput == null || cachedIndexInput.isClosed()) { logger.trace("Transfer Manager - IndexInput closed or not in cache"); // Doesn't exist or is closed, either way create a new one - return new DelayedCreationCachedIndexInput(fileCache, blobStreamReader, blobFetchRequest); + return new DelayedCreationCachedIndexInput(fileCache, streamReader, blobFetchRequest); } else { logger.trace("Transfer Manager - Already in cache"); // already in the cache and ready to be used (open) @@ -86,7 +87,7 @@ public IndexInput fetchBlob(BlobFetchRequest blobFetchRequest) throws IOExceptio } @SuppressWarnings("removal") - private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobStreamReader blobStreamReader, BlobFetchRequest request) { + private static FileCachedIndexInput createIndexInput(FileCache fileCache, StreamReader streamReader, BlobFetchRequest request) { // We need to do a privileged action here in order to fetch from remote // and write to the local file cache in case this is invoked as a side // effect of a plugin (such as a scripted search) that doesn't have the @@ -101,7 +102,7 @@ private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobSt ) { for (BlobFetchRequest.BlobPart blobPart : request.blobParts()) { try ( - InputStream snapshotFileInputStream = blobStreamReader.read( + InputStream snapshotFileInputStream = streamReader.read( blobPart.getBlobName(), blobPart.getPosition(), blobPart.getLength() @@ -129,15 +130,15 @@ private static FileCachedIndexInput createIndexInput(FileCache fileCache, BlobSt */ private static class DelayedCreationCachedIndexInput implements CachedIndexInput { private final FileCache fileCache; - private final BlobStreamReader blobStreamReader; + private final StreamReader streamReader; private final BlobFetchRequest request; private final CompletableFuture result = new CompletableFuture<>(); private final AtomicBoolean isStarted = new AtomicBoolean(false); private final AtomicBoolean isClosed = new AtomicBoolean(false); - private DelayedCreationCachedIndexInput(FileCache fileCache, BlobStreamReader blobStreamReader, BlobFetchRequest request) { + private DelayedCreationCachedIndexInput(FileCache fileCache, StreamReader streamReader, BlobFetchRequest request) { this.fileCache = fileCache; - this.blobStreamReader = blobStreamReader; + this.streamReader = streamReader; this.request = request; } @@ -149,7 +150,7 @@ public IndexInput getIndexInput() throws IOException { if (isStarted.getAndSet(true) == false) { // We're the first one here, need to download the block try { - result.complete(createIndexInput(fileCache, blobStreamReader, request)); + result.complete(createIndexInput(fileCache, streamReader, request)); } catch (Exception e) { result.completeExceptionally(e); fileCache.remove(request.getFilePath()); diff --git a/server/src/main/java/org/opensearch/indices/IndicesService.java b/server/src/main/java/org/opensearch/indices/IndicesService.java index aef40b80ea636..a7d879fc06981 100644 --- a/server/src/main/java/org/opensearch/indices/IndicesService.java +++ b/server/src/main/java/org/opensearch/indices/IndicesService.java @@ -443,7 +443,6 @@ public void onRemoval(ShardId shardId, String fieldName, boolean wasEvicted, lon this.directoryFactories = directoryFactories; this.recoveryStateFactories = recoveryStateFactories; - this.fileCache = fileCache; // doClose() is called when shutting down a node, yet there might still be ongoing requests // that we need to wait for before closing some resources such as the caches. In order to // avoid closing these resources while ongoing requests are still being processed, we use a @@ -499,6 +498,68 @@ protected void closeInternal() { .addSettingsUpdateConsumer(CLUSTER_DEFAULT_INDEX_REFRESH_INTERVAL_SETTING, this::onRefreshIntervalUpdate); this.recoverySettings = recoverySettings; this.remoteStoreSettings = remoteStoreSettings; + this.fileCache = fileCache; + } + + public IndicesService( + Settings settings, + PluginsService pluginsService, + NodeEnvironment nodeEnv, + NamedXContentRegistry xContentRegistry, + AnalysisRegistry analysisRegistry, + IndexNameExpressionResolver indexNameExpressionResolver, + MapperRegistry mapperRegistry, + NamedWriteableRegistry namedWriteableRegistry, + ThreadPool threadPool, + IndexScopedSettings indexScopedSettings, + CircuitBreakerService circuitBreakerService, + BigArrays bigArrays, + ScriptService scriptService, + ClusterService clusterService, + Client client, + MetaStateService metaStateService, + Collection>> engineFactoryProviders, + Map directoryFactories, + ValuesSourceRegistry valuesSourceRegistry, + Map recoveryStateFactories, + IndexStorePlugin.DirectoryFactory remoteDirectoryFactory, + Supplier repositoriesServiceSupplier, + SearchRequestStats searchRequestStats, + @Nullable RemoteStoreStatsTrackerFactory remoteStoreStatsTrackerFactory, + RecoverySettings recoverySettings, + CacheService cacheService, + RemoteStoreSettings remoteStoreSettings + ) { + this( + settings, + pluginsService, + nodeEnv, + xContentRegistry, + analysisRegistry, + indexNameExpressionResolver, + mapperRegistry, + namedWriteableRegistry, + threadPool, + indexScopedSettings, + circuitBreakerService, + bigArrays, + scriptService, + clusterService, + client, + metaStateService, + engineFactoryProviders, + directoryFactories, + valuesSourceRegistry, + recoveryStateFactories, + remoteDirectoryFactory, + repositoriesServiceSupplier, + searchRequestStats, + remoteStoreStatsTrackerFactory, + recoverySettings, + cacheService, + remoteStoreSettings, + null + ); } /** diff --git a/server/src/test/java/org/opensearch/index/IndexModuleTests.java b/server/src/test/java/org/opensearch/index/IndexModuleTests.java index 6eddd8c4a9b4a..4ce4936c047d9 100644 --- a/server/src/test/java/org/opensearch/index/IndexModuleTests.java +++ b/server/src/test/java/org/opensearch/index/IndexModuleTests.java @@ -278,8 +278,7 @@ public void testWrapperIsBound() throws IOException { Collections.emptyMap(), () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - Collections.emptyMap(), - null + Collections.emptyMap() ); module.setReaderWrapper(s -> new Wrapper()); @@ -305,8 +304,7 @@ public void testRegisterIndexStore() throws IOException { indexStoreFactories, () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - Collections.emptyMap(), - null + Collections.emptyMap() ); final IndexService indexService = newIndexService(module); @@ -634,8 +632,7 @@ public void testRegisterCustomRecoveryStateFactory() throws IOException { Collections.emptyMap(), () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - recoveryStateFactories, - null + recoveryStateFactories ); final IndexService indexService = newIndexService(module); @@ -667,8 +664,7 @@ private static IndexModule createIndexModule(IndexSettings indexSettings, Analys Collections.emptyMap(), () -> true, new IndexNameExpressionResolver(new ThreadContext(Settings.EMPTY)), - Collections.emptyMap(), - null + Collections.emptyMap() ); } diff --git a/server/src/test/java/org/opensearch/index/store/BaseRemoteSegmentStoreDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/BaseRemoteSegmentStoreDirectoryTests.java new file mode 100644 index 0000000000000..ff9b62a341deb --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/BaseRemoteSegmentStoreDirectoryTests.java @@ -0,0 +1,178 @@ +/* + * 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.store; + +import org.apache.lucene.index.SegmentInfos; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.concurrent.OpenSearchExecutors; +import org.opensearch.index.engine.NRTReplicationEngineFactory; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.shard.IndexShardTestCase; +import org.opensearch.index.store.lockmanager.RemoteStoreMetadataLockManager; +import org.opensearch.indices.replication.common.ReplicationType; +import org.opensearch.threadpool.ThreadPool; +import org.junit.After; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; + +import static org.opensearch.index.store.RemoteSegmentStoreDirectory.METADATA_FILES_TO_FETCH; +import static org.opensearch.test.RemoteStoreTestUtils.createMetadataFileBytes; +import static org.opensearch.test.RemoteStoreTestUtils.getDummyMetadata; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class BaseRemoteSegmentStoreDirectoryTests extends IndexShardTestCase { + + protected RemoteDirectory remoteDataDirectory; + protected RemoteDirectory remoteMetadataDirectory; + protected RemoteStoreMetadataLockManager mdLockManager; + protected RemoteSegmentStoreDirectory remoteSegmentStoreDirectory; + protected TestUploadListener testUploadTracker; + protected IndexShard indexShard; + protected SegmentInfos segmentInfos; + protected ThreadPool threadPool; + + protected final String metadataFilename = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( + 12, + 23, + 34, + 1, + 1, + "node-1" + ); + + protected final String metadataFilenameDup = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( + 12, + 23, + 34, + 2, + 1, + "node-2" + ); + protected final String metadataFilename2 = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( + 12, + 13, + 34, + 1, + 1, + "node-1" + ); + protected final String metadataFilename3 = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( + 10, + 38, + 34, + 1, + 1, + "node-1" + ); + protected final String metadataFilename4 = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( + 10, + 36, + 34, + 1, + 1, + "node-1" + ); + + public void setupRemoteSegmentStoreDirectory() throws IOException { + remoteDataDirectory = mock(RemoteDirectory.class); + remoteMetadataDirectory = mock(RemoteDirectory.class); + mdLockManager = mock(RemoteStoreMetadataLockManager.class); + threadPool = mock(ThreadPool.class); + testUploadTracker = new TestUploadListener(); + + 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()); + remoteSegmentStoreDirectory = new RemoteSegmentStoreDirectory( + remoteDataDirectory, + remoteMetadataDirectory, + mdLockManager, + threadPool, + indexShard.shardId() + ); + try (Store store = indexShard.store()) { + segmentInfos = store.readLastCommittedSegmentsInfo(); + } + + when(threadPool.executor(ThreadPool.Names.REMOTE_PURGE)).thenReturn(executorService); + when(threadPool.executor(ThreadPool.Names.REMOTE_RECOVERY)).thenReturn(executorService); + when(threadPool.executor(ThreadPool.Names.SAME)).thenReturn(executorService); + } + + protected Map> populateMetadata() throws IOException { + List metadataFiles = new ArrayList<>(); + + metadataFiles.add(metadataFilename); + metadataFiles.add(metadataFilename2); + metadataFiles.add(metadataFilename3); + + when( + remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( + RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, + METADATA_FILES_TO_FETCH + ) + ).thenReturn(List.of(metadataFilename)); + when( + remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( + RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, + Integer.MAX_VALUE + ) + ).thenReturn(metadataFiles); + + Map> metadataFilenameContentMapping = Map.of( + metadataFilename, + getDummyMetadata("_0", 1), + metadataFilename2, + getDummyMetadata("_0", 1), + metadataFilename3, + getDummyMetadata("_0", 1) + ); + + when(remoteMetadataDirectory.getBlobStream(metadataFilename)).thenAnswer( + I -> createMetadataFileBytes( + metadataFilenameContentMapping.get(metadataFilename), + indexShard.getLatestReplicationCheckpoint(), + segmentInfos + ) + ); + when(remoteMetadataDirectory.getBlobStream(metadataFilename2)).thenAnswer( + I -> createMetadataFileBytes( + metadataFilenameContentMapping.get(metadataFilename2), + indexShard.getLatestReplicationCheckpoint(), + segmentInfos + ) + ); + when(remoteMetadataDirectory.getBlobStream(metadataFilename3)).thenAnswer( + I -> createMetadataFileBytes( + metadataFilenameContentMapping.get(metadataFilename3), + indexShard.getLatestReplicationCheckpoint(), + segmentInfos + ) + ); + + return metadataFilenameContentMapping; + } + + @After + public void tearDown() throws Exception { + indexShard.close("test tearDown", true, false); + super.tearDown(); + } + +} diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java new file mode 100644 index 0000000000000..56d12537bf679 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -0,0 +1,189 @@ +/* + * 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.store; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; +import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; +import org.opensearch.index.store.remote.filecache.CachedIndexInput; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.filecache.NonBlockCachedIndexInput; +import org.junit.Before; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.startsWith; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) +public class CompositeDirectoryTests extends BaseRemoteSegmentStoreDirectoryTests { + private FileCache fileCache; + private FSDirectory localDirectory; + private CompositeDirectory compositeDirectory; + + @Before + public void setup() throws IOException { + setupRemoteSegmentStoreDirectory(); + localDirectory = mock(FSDirectory.class); + fileCache = mock(FileCache.class); + compositeDirectory = new CompositeDirectory(localDirectory, remoteSegmentStoreDirectory, fileCache); + } + + public void testListAll() throws IOException { + populateMetadata(); + when(localDirectory.listAll()).thenReturn(new String[] { "_1.cfe", "_2.cfe", "_0.cfe_block_7", "_0.cfs_block_7" }); + + String[] actualFileNames = compositeDirectory.listAll(); + String[] expectedFileNames = new String[] { "_0.cfe", "_0.cfs", "_0.si", "_1.cfe", "_2.cfe", "segments_1" }; + assertArrayEquals(expectedFileNames, actualFileNames); + } + + public void testDeleteFile() throws IOException { + Path basePath = mock(Path.class); + Path resolvedPath = mock(Path.class); + when(basePath.resolve("_0.si")).thenReturn(resolvedPath); + when(localDirectory.getDirectory()).thenReturn(basePath); + + compositeDirectory.deleteFile("_0.si"); + verify(fileCache).remove(resolvedPath); + } + + public void testFileLength() throws IOException { + populateMetadata(); + remoteSegmentStoreDirectory.init(); + Path basePath = mock(Path.class); + Path resolvedPath = mock(Path.class); + when(basePath.resolve("_0.si")).thenReturn(resolvedPath); + when(localDirectory.getDirectory()).thenReturn(basePath); + when(localDirectory.fileLength("_0.si")).thenReturn(7L); + + // File present locally + CachedIndexInput indexInput = mock(CachedIndexInput.class); + when(fileCache.get(resolvedPath)).thenReturn(indexInput); + assertEquals(compositeDirectory.fileLength("_0.si"), 7L); + verify(localDirectory).fileLength(startsWith("_0.si")); + + // File not present locally + Map uploadedSegments = remoteSegmentStoreDirectory + .getSegmentsUploadedToRemoteStore(); + assertTrue(uploadedSegments.containsKey("_0.si")); + when(fileCache.get(resolvedPath)).thenReturn(null); + assertEquals(compositeDirectory.fileLength("_0.si"), uploadedSegments.get("_0.si").getLength()); + } + + public void testCreateOutput() throws IOException { + IndexOutput indexOutput = mock(IndexOutput.class); + when(localDirectory.createOutput("_0.si", IOContext.DEFAULT)).thenReturn(indexOutput); + IndexOutput actualIndexOutput = compositeDirectory.createOutput("_0.si", IOContext.DEFAULT); + assert actualIndexOutput instanceof CloseableFilterIndexOutput; + verify(localDirectory).createOutput("_0.si", IOContext.DEFAULT); + } + + public void testCreateTempOutput() throws IOException { + IndexOutput indexOutput = mock(IndexOutput.class); + when(localDirectory.createTempOutput("prefix", "suffix", IOContext.DEFAULT)).thenReturn(indexOutput); + compositeDirectory.createTempOutput("prefix", "suffix", IOContext.DEFAULT); + verify(localDirectory).createTempOutput("prefix", "suffix", IOContext.DEFAULT); + } + + public void testSync() throws IOException { + populateMetadata(); + remoteSegmentStoreDirectory.init(); + Collection names = List.of("_0.cfe", "_0.cfs", "_1.cfe", "_1.cfs", "_2.nvm", "segments_1"); + compositeDirectory.sync(names); + verify(localDirectory).sync(List.of("_1.cfe", "_1.cfs", "_2.nvm")); + } + + public void testSyncMetaData() throws IOException { + compositeDirectory.syncMetaData(); + verify(localDirectory).syncMetaData(); + } + + public void testRename() throws IOException { + Path basePath = mock(Path.class); + Path resolvedPathOldFile = mock(Path.class); + Path resolvedPathNewFile = mock(Path.class); + when(basePath.resolve("old_file_name")).thenReturn(resolvedPathOldFile); + when(basePath.resolve("new_file_name")).thenReturn(resolvedPathNewFile); + when(localDirectory.getDirectory()).thenReturn(basePath); + CachedIndexInput indexInput = mock(CachedIndexInput.class); + when(fileCache.get(resolvedPathNewFile)).thenReturn(indexInput); + compositeDirectory.rename("old_file_name", "new_file_name"); + verify(localDirectory).rename("old_file_name", "new_file_name"); + verify(fileCache).remove(resolvedPathOldFile); + verify(fileCache).put(eq(resolvedPathNewFile), any(NonBlockCachedIndexInput.class)); + } + + public void testOpenInput() throws IOException { + populateMetadata(); + remoteSegmentStoreDirectory.init(); + Path basePath = mock(Path.class); + Path resolvedPathInCache = mock(Path.class); + Path resolvedPathNotInCache = mock(Path.class); + when(basePath.resolve("_0.si")).thenReturn(resolvedPathInCache); + when(basePath.resolve("_0.cfs")).thenReturn(resolvedPathNotInCache); + when(localDirectory.getDirectory()).thenReturn(basePath); + CachedIndexInput cachedIndexInput = mock(CachedIndexInput.class); + IndexInput localIndexInput = mock(IndexInput.class); + IndexInput indexInput = mock(IndexInput.class); + when(fileCache.get(resolvedPathInCache)).thenReturn(cachedIndexInput); + when(fileCache.compute(eq(resolvedPathInCache), any())).thenReturn(cachedIndexInput); + when(cachedIndexInput.getIndexInput()).thenReturn(indexInput); + when(indexInput.clone()).thenReturn(indexInput); + when(fileCache.get(resolvedPathNotInCache)).thenReturn(null); + + // Temp file, read directly form local directory + when(localDirectory.openInput("_0.tmp", IOContext.DEFAULT)).thenReturn(localIndexInput); + assertEquals(compositeDirectory.openInput("_0.tmp", IOContext.DEFAULT), localIndexInput); + verify(localDirectory).openInput("_0.tmp", IOContext.DEFAULT); + + // File present in file cache + assertEquals(compositeDirectory.openInput("_0.si", IOContext.DEFAULT), indexInput); + + // File present in Remote + IndexInput indexInput1 = compositeDirectory.openInput("_0.cfs", IOContext.DEFAULT); + assert indexInput1 instanceof OnDemandBlockSnapshotIndexInput; + } + + public void testClose() throws IOException { + Path basePath = mock(Path.class); + Path resolvedPath1 = mock(Path.class); + Path resolvedPath2 = mock(Path.class); + when(basePath.resolve("_0.si")).thenReturn(resolvedPath1); + when(basePath.resolve("_0.cfs")).thenReturn(resolvedPath2); + when(localDirectory.getDirectory()).thenReturn(basePath); + when(localDirectory.listAll()).thenReturn(new String[] { "_0.si", "_0.cfs" }); + compositeDirectory.close(); + verify(localDirectory).close(); + verify(fileCache).remove(resolvedPath1); + verify(fileCache).remove(resolvedPath2); + } + + public void testGetPendingDeletions() throws IOException { + Set pendingDeletions = new HashSet<>(Arrays.asList("_0.si", "_0.cfs", "_0.cfe")); + when(localDirectory.getPendingDeletions()).thenReturn(pendingDeletions); + assertEquals(pendingDeletions, compositeDirectory.getPendingDeletions()); + } +} 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 567199cf64cd8..574c5bf620474 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteSegmentStoreDirectoryTests.java @@ -10,7 +10,6 @@ import org.apache.logging.log4j.Level; import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexFormatTooNewException; @@ -23,34 +22,25 @@ import org.apache.lucene.store.OutputStreamIndexOutput; import org.apache.lucene.tests.util.LuceneTestCase; import org.apache.lucene.util.Version; -import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.UUIDs; import org.opensearch.common.blobstore.AsyncMultiStreamBlobContainer; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.io.VersionedCodecStreamWrapper; import org.opensearch.common.io.stream.BytesStreamOutput; import org.opensearch.common.lucene.store.ByteArrayIndexInput; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.concurrent.OpenSearchExecutors; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.bytes.BytesReference; import org.opensearch.core.index.Index; import org.opensearch.core.index.shard.ShardId; -import org.opensearch.index.engine.NRTReplicationEngineFactory; import org.opensearch.index.remote.RemoteStoreEnums.PathHashAlgorithm; import org.opensearch.index.remote.RemoteStoreEnums.PathType; import org.opensearch.index.remote.RemoteStorePathStrategy; import org.opensearch.index.remote.RemoteStoreUtils; -import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.shard.IndexShardTestCase; -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.common.ReplicationType; import org.opensearch.test.MockLogAppender; import org.opensearch.test.junit.annotations.TestLogging; import org.opensearch.threadpool.ThreadPool; -import org.junit.After; import org.junit.Before; import java.io.ByteArrayInputStream; @@ -64,7 +54,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -87,95 +76,11 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -public class RemoteSegmentStoreDirectoryTests extends IndexShardTestCase { - private static final Logger logger = LogManager.getLogger(RemoteSegmentStoreDirectoryTests.class); - private RemoteDirectory remoteDataDirectory; - private RemoteDirectory remoteMetadataDirectory; - private RemoteStoreMetadataLockManager mdLockManager; - - private RemoteSegmentStoreDirectory remoteSegmentStoreDirectory; - private TestUploadListener testUploadTracker; - private IndexShard indexShard; - private SegmentInfos segmentInfos; - private ThreadPool threadPool; - - private final String metadataFilename = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( - 12, - 23, - 34, - 1, - 1, - "node-1" - ); - - private final String metadataFilenameDup = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( - 12, - 23, - 34, - 2, - 1, - "node-2" - ); - private final String metadataFilename2 = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( - 12, - 13, - 34, - 1, - 1, - "node-1" - ); - private final String metadataFilename3 = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( - 10, - 38, - 34, - 1, - 1, - "node-1" - ); - private final String metadataFilename4 = RemoteSegmentStoreDirectory.MetadataFilenameUtils.getMetadataFilename( - 10, - 36, - 34, - 1, - 1, - "node-1" - ); +public class RemoteSegmentStoreDirectoryTests extends BaseRemoteSegmentStoreDirectoryTests { @Before public void setup() throws IOException { - remoteDataDirectory = mock(RemoteDirectory.class); - remoteMetadataDirectory = mock(RemoteDirectory.class); - mdLockManager = mock(RemoteStoreMetadataLockManager.class); - threadPool = mock(ThreadPool.class); - testUploadTracker = new TestUploadListener(); - - 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()); - remoteSegmentStoreDirectory = new RemoteSegmentStoreDirectory( - remoteDataDirectory, - remoteMetadataDirectory, - mdLockManager, - threadPool, - indexShard.shardId() - ); - try (Store store = indexShard.store()) { - segmentInfos = store.readLastCommittedSegmentsInfo(); - } - - when(threadPool.executor(ThreadPool.Names.REMOTE_PURGE)).thenReturn(executorService); - when(threadPool.executor(ThreadPool.Names.REMOTE_RECOVERY)).thenReturn(executorService); - when(threadPool.executor(ThreadPool.Names.SAME)).thenReturn(executorService); - } - - @After - public void tearDown() throws Exception { - indexShard.close("test tearDown", true, false); - super.tearDown(); + setupRemoteSegmentStoreDirectory(); } public void testUploadedSegmentMetadataToString() { @@ -256,60 +161,6 @@ public void testInitMultipleMetadataFile() throws IOException { assertThrows(IllegalStateException.class, () -> remoteSegmentStoreDirectory.init()); } - private Map> populateMetadata() throws IOException { - List metadataFiles = new ArrayList<>(); - - metadataFiles.add(metadataFilename); - metadataFiles.add(metadataFilename2); - metadataFiles.add(metadataFilename3); - - when( - remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, - METADATA_FILES_TO_FETCH - ) - ).thenReturn(List.of(metadataFilename)); - when( - remoteMetadataDirectory.listFilesByPrefixInLexicographicOrder( - RemoteSegmentStoreDirectory.MetadataFilenameUtils.METADATA_PREFIX, - Integer.MAX_VALUE - ) - ).thenReturn(metadataFiles); - - Map> metadataFilenameContentMapping = Map.of( - metadataFilename, - getDummyMetadata("_0", 1), - metadataFilename2, - getDummyMetadata("_0", 1), - metadataFilename3, - getDummyMetadata("_0", 1) - ); - - when(remoteMetadataDirectory.getBlobStream(metadataFilename)).thenAnswer( - I -> createMetadataFileBytes( - metadataFilenameContentMapping.get(metadataFilename), - indexShard.getLatestReplicationCheckpoint(), - segmentInfos - ) - ); - when(remoteMetadataDirectory.getBlobStream(metadataFilename2)).thenAnswer( - I -> createMetadataFileBytes( - metadataFilenameContentMapping.get(metadataFilename2), - indexShard.getLatestReplicationCheckpoint(), - segmentInfos - ) - ); - when(remoteMetadataDirectory.getBlobStream(metadataFilename3)).thenAnswer( - I -> createMetadataFileBytes( - metadataFilenameContentMapping.get(metadataFilename3), - indexShard.getLatestReplicationCheckpoint(), - segmentInfos - ) - ); - - return metadataFilenameContentMapping; - } - public void testInit() throws IOException { populateMetadata(); diff --git a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java index b035e117ff3ee..9c58fc8fde084 100644 --- a/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java +++ b/server/src/test/java/org/opensearch/snapshots/SnapshotResiliencyTests.java @@ -2078,8 +2078,7 @@ public void onFailure(final Exception e) { new RemoteStoreStatsTrackerFactory(clusterService, settings), DefaultRecoverySettings.INSTANCE, new CacheModule(new ArrayList<>(), settings).getCacheService(), - DefaultRemoteStoreSettings.INSTANCE, - null + DefaultRemoteStoreSettings.INSTANCE ); final RecoverySettings recoverySettings = new RecoverySettings(settings, clusterSettings); snapshotShardsService = new SnapshotShardsService( diff --git a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java index 0eca08a7678ae..71ab56c98312a 100644 --- a/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java +++ b/test/framework/src/main/java/org/opensearch/test/OpenSearchIntegTestCase.java @@ -2077,6 +2077,10 @@ protected boolean addMockTransportService() { return true; } + protected boolean addMockIndexStorePlugin() { + return true; + } + /** Returns {@code true} iff this test cluster should use a dummy http transport */ protected boolean addMockHttpTransport() { return true; @@ -2119,7 +2123,7 @@ protected Collection> getMockPlugins() { if (randomBoolean() && addMockTransportService()) { mocks.add(MockTransportService.TestPlugin.class); } - if (randomBoolean()) { + if (randomBoolean() && addMockIndexStorePlugin()) { mocks.add(MockFSIndexStore.TestPlugin.class); } if (randomBoolean()) { From 723aba071daa239956bc6ba4697038d30602c5d9 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Thu, 9 May 2024 20:18:07 +0530 Subject: [PATCH 10/22] Add experimental annotations for newly created classes and review comment fixes Signed-off-by: Shreyansh Ray --- CHANGELOG.md | 1 + .../store/CloseableFilterIndexOutput.java | 10 ++++--- .../index/store/CompositeDirectory.java | 23 +++++++-------- .../filecache/FileCachedIndexInput.java | 5 ++-- ...put.java => FullFileCachedIndexInput.java} | 23 +++++++++++---- .../store/remote/utils/BlockIOContext.java | 29 +++++++------------ .../index/store/remote/utils/FileType.java | 19 +++++++++--- .../index/store/CompositeDirectoryTests.java | 4 +-- 8 files changed, 64 insertions(+), 50 deletions(-) rename server/src/main/java/org/opensearch/index/store/remote/filecache/{NonBlockCachedIndexInput.java => FullFileCachedIndexInput.java} (61%) diff --git a/CHANGELOG.md b/CHANGELOG.md index 8c3e63e36bc82..8c5aa4be0c72d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,6 +8,7 @@ The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), - Add fingerprint ingest processor ([#13724](https://github.com/opensearch-project/OpenSearch/pull/13724)) - [Remote Store] Rate limiter for remote store low priority uploads ([#14374](https://github.com/opensearch-project/OpenSearch/pull/14374/)) - Apply the date histogram rewrite optimization to range aggregation ([#13865](https://github.com/opensearch-project/OpenSearch/pull/13865)) +- [Writable Warm] Add composite directory implementation and integrate it with FileCache ([12782](https://github.com/opensearch-project/OpenSearch/pull/12782)) ### Dependencies - Bump `org.gradle.test-retry` from 1.5.8 to 1.5.9 ([#13442](https://github.com/opensearch-project/OpenSearch/pull/13442)) diff --git a/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java index 8df5d648b91f3..3a4309fe6ee6d 100644 --- a/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java +++ b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java @@ -9,6 +9,7 @@ package org.opensearch.index.store; import org.apache.lucene.store.IndexOutput; +import org.opensearch.common.annotation.ExperimentalApi; import org.opensearch.common.lucene.store.FilterIndexOutput; import java.io.IOException; @@ -16,8 +17,9 @@ /** * FilterIndexOutput which takes in an additional FunctionalInterface as a parameter to perform required operations once the IndexOutput is closed * - * @opensearch.internal + * @opensearch.experimental */ +@ExperimentalApi public class CloseableFilterIndexOutput extends FilterIndexOutput { /** @@ -25,11 +27,11 @@ public class CloseableFilterIndexOutput extends FilterIndexOutput { */ @FunctionalInterface public interface OnCloseListener { - void onClose(String name); + void onClose(String name) throws IOException; } - OnCloseListener onCloseListener; - String fileName; + private final OnCloseListener onCloseListener; + private final String fileName; public CloseableFilterIndexOutput(IndexOutput out, String fileName, OnCloseListener onCloseListener) { super("CloseableFilterIndexOutput for file " + fileName, out); diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 5536be1212ccb..f66120c5753f7 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -22,7 +22,7 @@ import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.filecache.NonBlockCachedIndexInput; +import org.opensearch.index.store.remote.filecache.FullFileCachedIndexInput; import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.utils.BlockIOContext; import org.opensearch.index.store.remote.utils.FileType; @@ -45,7 +45,7 @@ * All such abstractions will be handled by the Composite directory itself * Implements all required methods by Directory abstraction * - * @opensearch.internal + * @opensearch.experimental */ @ExperimentalApi public class CompositeDirectory extends FilterDirectory { @@ -143,9 +143,12 @@ public long fileLength(String name) throws IOException { long fileLength; Path key = localDirectory.getDirectory().resolve(name); if (isTempFile(name) || fileCache.get(key) != null) { - fileLength = localDirectory.fileLength(name); - fileCache.decRef(key); - logger.trace("fileLength from Local {}", fileLength); + try { + fileLength = localDirectory.fileLength(name); + logger.trace("fileLength from Local {}", fileLength); + } finally { + fileCache.decRef(key); + } } else { fileLength = remoteDirectory.fileLength(name); logger.trace("fileLength from Remote {}", fileLength); @@ -170,13 +173,7 @@ public IndexOutput createOutput(String name, IOContext context) throws IOExcepti /* * The CloseableFilterIndexOutput will ensure that the file is added to FileCache once write is completed on this file */ - return new CloseableFilterIndexOutput(localDirectory.createOutput(name, context), name, (fileName) -> { - try { - cacheFile(fileName); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); + return new CloseableFilterIndexOutput(localDirectory.createOutput(name, context), name, this::cacheFile); } finally { writeLock.unlock(); } @@ -393,7 +390,7 @@ private String[] getRemoteFiles() throws IOException { private void cacheFile(String name) throws IOException { Path filePath = localDirectory.getDirectory().resolve(name); - fileCache.put(filePath, new NonBlockCachedIndexInput(localDirectory.openInput(name, IOContext.READ))); + fileCache.put(filePath, new FullFileCachedIndexInput(fileCache, filePath, localDirectory.openInput(name, IOContext.READ))); // Decrementing ref here as above put call increments the ref of the key fileCache.decRef(filePath); // TODO : Pin the above filePath in the file cache once pinning support is added so that it cannot be evicted unless it has been diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java index 7d7c40be3a833..200a47e661ab4 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java @@ -133,8 +133,9 @@ public FileCachedIndexInput clone() { @Override public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { - // never reach here! - throw new UnsupportedOperationException("FileCachedIndexInput couldn't be sliced."); + IndexInput slicedIndexInput = luceneIndexInput.slice(sliceDescription, offset, length); + cache.incRef(filePath); + return new FileCachedIndexInput(cache, filePath, slicedIndexInput, true); } @Override diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/NonBlockCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java similarity index 61% rename from server/src/main/java/org/opensearch/index/store/remote/filecache/NonBlockCachedIndexInput.java rename to server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java index ee520dc22964e..7b1163b18727e 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/NonBlockCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java @@ -8,24 +8,35 @@ package org.opensearch.index.store.remote.filecache; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IndexInput; +import org.opensearch.common.annotation.ExperimentalApi; -import java.io.IOException; +import java.nio.file.Path; import java.util.concurrent.atomic.AtomicBoolean; /** * Implementation of the CachedIndexInput for NON_BLOCK files which takes in an IndexInput as parameter + * + * @opensearch.experimental */ -public class NonBlockCachedIndexInput implements CachedIndexInput { +@ExperimentalApi +public class FullFileCachedIndexInput implements CachedIndexInput { private final IndexInput indexInput; + private final FileCache fileCache; + private final Path path; + private final FileCachedIndexInput fileCachedIndexInput; private final AtomicBoolean isClosed; /** * Constructor - takes IndexInput as parameter */ - public NonBlockCachedIndexInput(IndexInput indexInput) { + public FullFileCachedIndexInput(FileCache fileCache, Path path, IndexInput indexInput) { + this.fileCache = fileCache; + this.path = path; this.indexInput = indexInput; + fileCachedIndexInput = new FileCachedIndexInput(fileCache, path, indexInput); isClosed = new AtomicBoolean(false); } @@ -33,8 +44,9 @@ public NonBlockCachedIndexInput(IndexInput indexInput) { * Returns the wrapped indexInput */ @Override - public IndexInput getIndexInput() throws IOException { - return indexInput; + public IndexInput getIndexInput() { + if (isClosed.get()) throw new AlreadyClosedException("Index input is already closed"); + return fileCachedIndexInput; } /** @@ -60,7 +72,6 @@ public boolean isClosed() { public void close() throws Exception { if (!isClosed.getAndSet(true)) { indexInput.close(); - isClosed.set(true); } } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java b/server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java index da94e4a46d307..a78dd85d6f194 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/BlockIOContext.java @@ -9,43 +9,29 @@ package org.opensearch.index.store.remote.utils; import org.apache.lucene.store.IOContext; +import org.opensearch.common.annotation.ExperimentalApi; /** * BlockIOContext is an extension of IOContext which can be used to pass block related information to the openInput() method of any directory + * + * @opensearch.experimental */ +@ExperimentalApi public class BlockIOContext extends IOContext { - private final boolean isBlockRequest; private long blockStart; private long blockSize; - /** - * Default constructor - */ - BlockIOContext(IOContext ctx) { - super(ctx.context); - this.isBlockRequest = false; - this.blockStart = -1; - this.blockSize = -1; - } - /** * Constructor to initialise BlockIOContext with block related information */ public BlockIOContext(IOContext ctx, long blockStart, long blockSize) { super(ctx.context); - this.isBlockRequest = true; + verifyBlockStartAndSize(blockStart, blockSize); this.blockStart = blockStart; this.blockSize = blockSize; } - /** - * Function to check if the Context contains a block request or not - */ - public boolean isBlockRequest() { - return isBlockRequest; - } - /** * Getter for blockStart */ @@ -59,4 +45,9 @@ public long getBlockStart() { public long getBlockSize() { return blockSize; } + + private void verifyBlockStartAndSize(long blockStart, long blockSize) { + if (blockStart < 0) throw new IllegalArgumentException("blockStart must be greater than or equal to 0"); + if (blockSize <= 0) throw new IllegalArgumentException(("blockSize must be greater than 0")); + } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java b/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java index 418f8a24a5f24..e340c82ba9ba3 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java @@ -8,18 +8,29 @@ package org.opensearch.index.store.remote.utils; +import org.opensearch.common.annotation.ExperimentalApi; + /** * Enum to represent whether a file is block or not + * + * @opensearch.experimental */ +@ExperimentalApi public enum FileType { /** * Block file */ - BLOCK, + BLOCK(".*_block_.*"), /** - * Non block file + * Full file - Non-Block */ - NON_BLOCK; + FULL(".*"); + + private final String pattern; + + FileType(String pattern) { + this.pattern = pattern; + } /** * Returns if the fileType is a block file or not @@ -32,7 +43,7 @@ public static boolean isBlockFile(FileType fileType) { * Returns if the fileName is block file or not */ public static boolean isBlockFile(String fileName) { - if (fileName.contains("_block_")) return true; + if (fileName.matches(FileType.BLOCK.pattern)) return true; return false; } } diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java index 56d12537bf679..16dd3ac7a68f7 100644 --- a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -18,7 +18,7 @@ import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.filecache.NonBlockCachedIndexInput; +import org.opensearch.index.store.remote.filecache.FullFileCachedIndexInput; import org.junit.Before; import java.io.IOException; @@ -133,7 +133,7 @@ public void testRename() throws IOException { compositeDirectory.rename("old_file_name", "new_file_name"); verify(localDirectory).rename("old_file_name", "new_file_name"); verify(fileCache).remove(resolvedPathOldFile); - verify(fileCache).put(eq(resolvedPathNewFile), any(NonBlockCachedIndexInput.class)); + verify(fileCache).put(eq(resolvedPathNewFile), any(FullFileCachedIndexInput.class)); } public void testOpenInput() throws IOException { From 959f90c2cab062fb8e6c37a565ab80a9dc66563e Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Thu, 9 May 2024 23:39:55 +0530 Subject: [PATCH 11/22] Use ref count as a temporary measure to prevent file from eviction until uploaded to Remote Signed-off-by: Shreyansh Ray --- .../opensearch/index/store/CompositeDirectory.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index f66120c5753f7..758a1f274820c 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -352,11 +352,13 @@ public void afterSyncToRemote(Collection files) throws IOException { for (String fileName : files) { writeLock.lock(); try { - /** + /* Decrementing the refCount here for the path so that it becomes eligible for eviction + * This is a temporary solution until pinning support is added * TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache * Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality */ logger.trace("File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", fileName); + fileCache.decRef(localDirectory.getDirectory().resolve(fileName)); // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); } finally { writeLock.unlock(); @@ -390,11 +392,12 @@ private String[] getRemoteFiles() throws IOException { private void cacheFile(String name) throws IOException { Path filePath = localDirectory.getDirectory().resolve(name); - fileCache.put(filePath, new FullFileCachedIndexInput(fileCache, filePath, localDirectory.openInput(name, IOContext.READ))); - // Decrementing ref here as above put call increments the ref of the key - fileCache.decRef(filePath); + // put will increase the refCount for the path, making sure it is not evicted, wil decrease the ref after it is uploaded to Remote + // so that it can be evicted after that + // this is just a temporary solution, will pin the file once support for that is added in FileCache // TODO : Pin the above filePath in the file cache once pinning support is added so that it cannot be evicted unless it has been // successfully uploaded to Remote + fileCache.put(filePath, new FullFileCachedIndexInput(fileCache, filePath, localDirectory.openInput(name, IOContext.READ))); } } From 2632738a887df6265a7d973b9db0274932bdafd9 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Fri, 10 May 2024 15:56:58 +0530 Subject: [PATCH 12/22] Remove method level locks Signed-off-by: Shreyansh Ray --- .../index/store/CompositeDirectory.java | 279 ++++++------------ .../index/store/CompositeDirectoryTests.java | 21 -- 2 files changed, 84 insertions(+), 216 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 758a1f274820c..583a181c4797a 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -36,7 +36,6 @@ import java.util.Collection; import java.util.HashSet; import java.util.Set; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; /** @@ -54,9 +53,6 @@ public class CompositeDirectory extends FilterDirectory { private final RemoteSegmentStoreDirectory remoteDirectory; private final FileCache fileCache; private final TransferManager transferManager; - private final ReentrantReadWriteLock readWriteLock = new ReentrantReadWriteLock(); - private final ReentrantReadWriteLock.WriteLock writeLock = readWriteLock.writeLock(); - private final ReentrantReadWriteLock.ReadLock readLock = readWriteLock.readLock(); /** * Constructor to initialise the composite directory @@ -87,25 +83,20 @@ public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirector @Override public String[] listAll() throws IOException { logger.trace("listAll() called"); - readLock.lock(); - try { - String[] localFiles = localDirectory.listAll(); - logger.trace("Local Directory files : {}", () -> Arrays.toString(localFiles)); - Set allFiles = new HashSet<>(Arrays.asList(localFiles)); - String[] remoteFiles = getRemoteFiles(); - allFiles.addAll(Arrays.asList(remoteFiles)); - logger.trace("Remote Directory files : {}", () -> Arrays.toString(remoteFiles)); - Set localLuceneFiles = allFiles.stream() - .filter(file -> !FileType.isBlockFile(file)) - .collect(Collectors.toUnmodifiableSet()); - String[] files = new String[localLuceneFiles.size()]; - localLuceneFiles.toArray(files); - Arrays.sort(files); - logger.trace("listAll() returns : {}", () -> Arrays.toString(files)); - return files; - } finally { - readLock.unlock(); - } + String[] localFiles = localDirectory.listAll(); + logger.trace("Local Directory files : {}", () -> Arrays.toString(localFiles)); + Set allFiles = new HashSet<>(Arrays.asList(localFiles)); + String[] remoteFiles = getRemoteFiles(); + allFiles.addAll(Arrays.asList(remoteFiles)); + logger.trace("Remote Directory files : {}", () -> Arrays.toString(remoteFiles)); + Set localLuceneFiles = allFiles.stream() + .filter(file -> !FileType.isBlockFile(file)) + .collect(Collectors.toUnmodifiableSet()); + String[] files = new String[localLuceneFiles.size()]; + localLuceneFiles.toArray(files); + Arrays.sort(files); + logger.trace("listAll() returns : {}", () -> Arrays.toString(files)); + return files; } /** @@ -117,16 +108,11 @@ public String[] listAll() throws IOException { @Override public void deleteFile(String name) throws IOException { logger.trace("deleteFile() called {}", name); - writeLock.lock(); - try { - /* - Not deleting from localDirectory directly since it causes a race condition when the localDirectory deletes a file, and it ends up in pendingDeletion state. - Meanwhile, fileCache on removal deletes the file directly via the Files class and later when the directory tries to delete the files pending for deletion (which happens before creating a new file), it causes NoSuchFileException and new file creation fails - */ - fileCache.remove(localDirectory.getDirectory().resolve(name)); - } finally { - writeLock.unlock(); - } + /* + Not deleting from localDirectory directly since it causes a race condition when the localDirectory deletes a file, and it ends up in pendingDeletion state. + Meanwhile, fileCache on removal deletes the file directly via the Files class and later when the directory tries to delete the files pending for deletion (which happens before creating a new file), it causes NoSuchFileException and new file creation fails + */ + fileCache.remove(localDirectory.getDirectory().resolve(name)); } /** @@ -138,25 +124,20 @@ Meanwhile, fileCache on removal deletes the file directly via the Files class an @Override public long fileLength(String name) throws IOException { logger.trace("fileLength() called {}", name); - readLock.lock(); - try { - long fileLength; - Path key = localDirectory.getDirectory().resolve(name); - if (isTempFile(name) || fileCache.get(key) != null) { - try { - fileLength = localDirectory.fileLength(name); - logger.trace("fileLength from Local {}", fileLength); - } finally { - fileCache.decRef(key); - } - } else { - fileLength = remoteDirectory.fileLength(name); - logger.trace("fileLength from Remote {}", fileLength); + long fileLength; + Path key = localDirectory.getDirectory().resolve(name); + if (isTempFile(name) || fileCache.get(key) != null) { + try { + fileLength = localDirectory.fileLength(name); + logger.trace("fileLength from Local {}", fileLength); + } finally { + fileCache.decRef(key); } - return fileLength; - } finally { - readLock.unlock(); + } else { + fileLength = remoteDirectory.fileLength(name); + logger.trace("fileLength from Remote {}", fileLength); } + return fileLength; } /** @@ -168,33 +149,8 @@ public long fileLength(String name) throws IOException { @Override public IndexOutput createOutput(String name, IOContext context) throws IOException { logger.trace("createOutput() called {}", name); - writeLock.lock(); - try { - /* - * The CloseableFilterIndexOutput will ensure that the file is added to FileCache once write is completed on this file - */ - return new CloseableFilterIndexOutput(localDirectory.createOutput(name, context), name, this::cacheFile); - } finally { - writeLock.unlock(); - } - } - - /** - * Creates a new, empty, temporary file in the directory and returns an {@link IndexOutput} - * instance for appending data to this file. - * - *

    The temporary file name (accessible via {@link IndexOutput#getName()}) will start with - * {@code prefix}, end with {@code suffix} and have a reserved file extension {@code .tmp}. - */ - @Override - public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException { - logger.trace("createTempOutput() called {} , {}", prefix, suffix); - writeLock.lock(); - try { - return localDirectory.createTempOutput(prefix, suffix, context); - } finally { - writeLock.unlock(); - } + // The CloseableFilterIndexOutput will ensure that the file is added to FileCache once write is completed on this file + return new CloseableFilterIndexOutput(localDirectory.createOutput(name, context), name, this::cacheFile); } /** @@ -204,32 +160,10 @@ public IndexOutput createTempOutput(String prefix, String suffix, IOContext cont @Override public void sync(Collection names) throws IOException { logger.trace("sync() called {}", names); - writeLock.lock(); - try { - Collection remoteFiles = Arrays.asList(getRemoteFiles()); - Collection filesToSync = names.stream() - .filter(name -> remoteFiles.contains(name) == false) - .collect(Collectors.toList()); - logger.trace("Synced files : {}", filesToSync); - localDirectory.sync(filesToSync); - } finally { - writeLock.unlock(); - } - } - - /** - * Ensures that directory metadata, such as recent file renames, are moved to stable storage. - * @throws IOException in case of I/O error - */ - @Override - public void syncMetaData() throws IOException { - logger.trace("syncMetaData() called "); - writeLock.lock(); - try { - localDirectory.syncMetaData(); - } finally { - writeLock.unlock(); - } + Collection remoteFiles = Arrays.asList(getRemoteFiles()); + Collection filesToSync = names.stream().filter(name -> remoteFiles.contains(name) == false).collect(Collectors.toList()); + logger.trace("Synced files : {}", filesToSync); + localDirectory.sync(filesToSync); } /** @@ -240,14 +174,9 @@ public void syncMetaData() throws IOException { @Override public void rename(String source, String dest) throws IOException { logger.trace("rename() called {}, {}", source, dest); - writeLock.lock(); - try { - localDirectory.rename(source, dest); - fileCache.remove(localDirectory.getDirectory().resolve(source)); - cacheFile(dest); - } finally { - writeLock.unlock(); - } + localDirectory.rename(source, dest); + fileCache.remove(localDirectory.getDirectory().resolve(source)); + cacheFile(dest); } /** @@ -259,48 +188,35 @@ public void rename(String source, String dest) throws IOException { @Override public IndexInput openInput(String name, IOContext context) throws IOException { logger.trace("openInput() called {}", name); - writeLock.lock(); - try { - /* - * We aren't tracking temporary files (created via createTempOutput) currently in FileCache as these are created and then deleted within a very short span of time - * We will be reading them directory from the local directory - */ - if (isTempFile(name)) { - return localDirectory.openInput(name, context); - } - /* - * Return directly from the FileCache (via TransferManager) if complete file is present - */ - - Path key = localDirectory.getDirectory().resolve(name); - CachedIndexInput indexInput = fileCache.get(key); - if (indexInput != null) { - logger.trace("Complete file found in FileCache"); - try { - return indexInput.getIndexInput().clone(); - } finally { - fileCache.decRef(key); - } - } - /* - * If file has been uploaded to the Remote Store, fetch it from the Remote Store in blocks via OnDemandCompositeBlockIndexInput - */ - else { - logger.trace("Complete file not in FileCache, to be fetched in Blocks from Remote"); - RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); - RemoteSegmentStoreDirectory.UploadedSegmentMetadata uploadedSegmentMetadata = remoteSegmentMetadata.getMetadata().get(name); - /* - * TODO : Refactor FileInfo and OnDemandBlockSnapshotIndexInput to more generic names as they are not Remote Snapshot specific - */ - BlobStoreIndexShardSnapshot.FileInfo fileInfo = new BlobStoreIndexShardSnapshot.FileInfo( - name, - new StoreFileMetadata(name, uploadedSegmentMetadata.getLength(), uploadedSegmentMetadata.getChecksum(), Version.LATEST), - null - ); - return new OnDemandBlockSnapshotIndexInput(fileInfo, localDirectory, transferManager); + // We aren't tracking temporary files (created via createTempOutput) currently in FileCache as these are created and then deleted + // within a very short span of time + // We will be reading them directory from the local directory + if (isTempFile(name)) { + return localDirectory.openInput(name, context); + } + // Return directly from the FileCache (via TransferManager) if complete file is present + Path key = localDirectory.getDirectory().resolve(name); + CachedIndexInput indexInput = fileCache.get(key); + if (indexInput != null) { + logger.trace("Complete file found in FileCache"); + try { + return indexInput.getIndexInput().clone(); + } finally { + fileCache.decRef(key); } - } finally { - writeLock.unlock(); + } + // If file has been uploaded to the Remote Store, fetch it from the Remote Store in blocks via OnDemandCompositeBlockIndexInput + else { + logger.trace("Complete file not in FileCache, to be fetched in Blocks from Remote"); + RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); + RemoteSegmentStoreDirectory.UploadedSegmentMetadata uploadedSegmentMetadata = remoteSegmentMetadata.getMetadata().get(name); + // TODO : Refactor FileInfo and OnDemandBlockSnapshotIndexInput to more generic names as they are not Remote Snapshot specific + BlobStoreIndexShardSnapshot.FileInfo fileInfo = new BlobStoreIndexShardSnapshot.FileInfo( + name, + new StoreFileMetadata(name, uploadedSegmentMetadata.getLength(), uploadedSegmentMetadata.getChecksum(), Version.LATEST), + null + ); + return new OnDemandBlockSnapshotIndexInput(fileInfo, localDirectory, transferManager); } } @@ -310,31 +226,9 @@ public IndexInput openInput(String name, IOContext context) throws IOException { */ @Override public void close() throws IOException { - writeLock.lock(); - try { - Arrays.stream(localDirectory.listAll()).forEach(f -> { - logger.trace("Removing file from cache {}", f); - fileCache.remove(localDirectory.getDirectory().resolve(f)); - }); - localDirectory.close(); - remoteDirectory.close(); - } finally { - writeLock.unlock(); - } - } - - /** - * Returns a set of files currently pending deletion in this directory. - * @throws IOException in case of I/O error - */ - @Override - public Set getPendingDeletions() throws IOException { - writeLock.lock(); - try { - return localDirectory.getPendingDeletions(); - } finally { - writeLock.unlock(); - } + Arrays.stream(localDirectory.listAll()).forEach(f -> fileCache.remove(localDirectory.getDirectory().resolve(f))); + localDirectory.close(); + remoteDirectory.close(); } /** @@ -350,19 +244,15 @@ public void afterSyncToRemote(Collection files) throws IOException { return; } for (String fileName : files) { - writeLock.lock(); - try { - /* Decrementing the refCount here for the path so that it becomes eligible for eviction - * This is a temporary solution until pinning support is added - * TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache - * Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality - */ - logger.trace("File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", fileName); - fileCache.decRef(localDirectory.getDirectory().resolve(fileName)); - // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); - } finally { - writeLock.unlock(); - } + /* + Decrementing the refCount here for the path so that it becomes eligible for eviction + This is a temporary solution until pinning support is added + TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache + Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality + */ + logger.trace("File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", fileName); + fileCache.decRef(localDirectory.getDirectory().resolve(fileName)); + // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); } } @@ -379,11 +269,10 @@ private String[] getRemoteFiles() throws IOException { remoteFiles = remoteDirectory.listAll(); } catch (NullPointerException e) { /* - * There are two scenarios where the listAll() call on remote directory returns NullPointerException: - * - When remote directory is not set - * - When init() of remote directory has not yet been called - * - * Returning an empty list in these scenarios + There are two scenarios where the listAll() call on remote directory returns NullPointerException: + - When remote directory is not set + - When init() of remote directory has not yet been called + Returning an empty list in the above scenarios */ remoteFiles = new String[0]; } diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java index 16dd3ac7a68f7..771734ac23a9e 100644 --- a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -23,12 +23,9 @@ import java.io.IOException; import java.nio.file.Path; -import java.util.Arrays; import java.util.Collection; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -101,13 +98,6 @@ public void testCreateOutput() throws IOException { verify(localDirectory).createOutput("_0.si", IOContext.DEFAULT); } - public void testCreateTempOutput() throws IOException { - IndexOutput indexOutput = mock(IndexOutput.class); - when(localDirectory.createTempOutput("prefix", "suffix", IOContext.DEFAULT)).thenReturn(indexOutput); - compositeDirectory.createTempOutput("prefix", "suffix", IOContext.DEFAULT); - verify(localDirectory).createTempOutput("prefix", "suffix", IOContext.DEFAULT); - } - public void testSync() throws IOException { populateMetadata(); remoteSegmentStoreDirectory.init(); @@ -116,11 +106,6 @@ public void testSync() throws IOException { verify(localDirectory).sync(List.of("_1.cfe", "_1.cfs", "_2.nvm")); } - public void testSyncMetaData() throws IOException { - compositeDirectory.syncMetaData(); - verify(localDirectory).syncMetaData(); - } - public void testRename() throws IOException { Path basePath = mock(Path.class); Path resolvedPathOldFile = mock(Path.class); @@ -180,10 +165,4 @@ public void testClose() throws IOException { verify(fileCache).remove(resolvedPath1); verify(fileCache).remove(resolvedPath2); } - - public void testGetPendingDeletions() throws IOException { - Set pendingDeletions = new HashSet<>(Arrays.asList("_0.si", "_0.cfs", "_0.cfe")); - when(localDirectory.getPendingDeletions()).thenReturn(pendingDeletions); - assertEquals(pendingDeletions, compositeDirectory.getPendingDeletions()); - } } From ba34798726323f17c2c37d155613b5a2de2508c3 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Fri, 10 May 2024 17:21:02 +0530 Subject: [PATCH 13/22] Handle tmp file deletion Signed-off-by: Shreyansh Ray --- .../opensearch/index/store/CompositeDirectory.java | 14 +++++++++----- .../index/store/CompositeDirectoryTests.java | 6 +++++- 2 files changed, 14 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 583a181c4797a..597a64158be03 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -108,11 +108,15 @@ public String[] listAll() throws IOException { @Override public void deleteFile(String name) throws IOException { logger.trace("deleteFile() called {}", name); - /* - Not deleting from localDirectory directly since it causes a race condition when the localDirectory deletes a file, and it ends up in pendingDeletion state. - Meanwhile, fileCache on removal deletes the file directly via the Files class and later when the directory tries to delete the files pending for deletion (which happens before creating a new file), it causes NoSuchFileException and new file creation fails - */ - fileCache.remove(localDirectory.getDirectory().resolve(name)); + if (isTempFile(name)) { + localDirectory.deleteFile(name); + } else { + /* + Not deleting from localDirectory directly since it causes a race condition when the localDirectory deletes a file, and it ends up in pendingDeletion state. + Meanwhile, fileCache on removal deletes the file directly via the Files class and later when the directory tries to delete the files pending for deletion (which happens before creating a new file), it causes NoSuchFileException and new file creation fails + */ + fileCache.remove(localDirectory.getDirectory().resolve(name)); + } } /** diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java index 771734ac23a9e..cfc6f0277d23d 100644 --- a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -28,6 +28,7 @@ import java.util.Map; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.startsWith; import static org.mockito.Mockito.mock; @@ -60,9 +61,12 @@ public void testListAll() throws IOException { public void testDeleteFile() throws IOException { Path basePath = mock(Path.class); Path resolvedPath = mock(Path.class); - when(basePath.resolve("_0.si")).thenReturn(resolvedPath); + when(basePath.resolve(anyString())).thenReturn(resolvedPath); when(localDirectory.getDirectory()).thenReturn(basePath); + compositeDirectory.deleteFile("_0.tmp"); + verify(localDirectory).deleteFile("_0.tmp"); + compositeDirectory.deleteFile("_0.si"); verify(fileCache).remove(resolvedPath); } From 85003eefda2da393d2680d47fe4d95b4b92e9508 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Mon, 13 May 2024 19:59:36 +0530 Subject: [PATCH 14/22] Nit fixes Signed-off-by: Shreyansh Ray --- .../remotestore/CompositeDirectoryIT.java | 3 +-- .../org/opensearch/index/IndexModule.java | 2 +- .../org/opensearch/index/IndexService.java | 21 +++++++++++++------ .../index/store/CompositeDirectory.java | 16 ++++++-------- .../filecache/FullFileCachedIndexInput.java | 7 ++----- .../index/store/CompositeDirectoryTests.java | 21 +++++++++++++++++-- 6 files changed, 44 insertions(+), 26 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java index a34b665561289..3d52e6614f6a3 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java @@ -26,7 +26,6 @@ import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; import org.opensearch.indices.IndicesService; import org.opensearch.test.OpenSearchIntegTestCase; -import org.opensearch.test.junit.annotations.TestLogging; import java.util.Map; @@ -36,7 +35,7 @@ @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) // Uncomment the below line to enable trace level logs for this test for better debugging -@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") +// @TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") public class CompositeDirectoryIT extends RemoteStoreBaseIntegTestCase { /* diff --git a/server/src/main/java/org/opensearch/index/IndexModule.java b/server/src/main/java/org/opensearch/index/IndexModule.java index d79f19f17e167..4c494a6b35153 100644 --- a/server/src/main/java/org/opensearch/index/IndexModule.java +++ b/server/src/main/java/org/opensearch/index/IndexModule.java @@ -646,7 +646,7 @@ public static DataLocalityType getValueOf(final String localityType) { if (type != null) { return type; } - throw new IllegalArgumentException("Unknown Locality Type constant [" + localityType + "]."); + throw new IllegalArgumentException("Unknown locality type constant [" + localityType + "]."); } } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 78bfedb7b6d65..d021b290e585b 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -618,14 +618,23 @@ public synchronized IndexShard createShard( // TODO : Need to remove this check after support for hot indices is added in Composite Directory this.indexSettings.isStoreLocalityPartial()) { /* - * Currently Composite Directory only supports local directory to be of type FSDirectory - * The reason is that FileCache currently has it key type as Path - * Composite Directory currently uses FSDirectory's getDirectory() method to fetch and use the Path for operating on FileCache - * TODO : Refactor FileCache to have key in form of String instead of Path. Once that is done we can remove this assertion + Currently Composite Directory only supports local directory to be of type FSDirectory + The reason is that FileCache currently has it key type as Path + Composite Directory currently uses FSDirectory's getDirectory() method to fetch and use the Path for operating on FileCache + TODO : Refactor FileCache to have key in form of String instead of Path. Once that is done we can remove this assertion */ Directory localDirectory = directoryFactory.newDirectory(this.indexSettings, path); - assert localDirectory instanceof FSDirectory : "For Composite Directory, local directory must be of type FSDirectory"; - assert fileCache != null : "File Cache not initialized on this Node, cannot create Composite Directory without FileCache"; + + if (localDirectory instanceof FSDirectory == false) throw new IllegalStateException( + "For Composite Directory, local directory must be of type FSDirectory" + ); + else if (fileCache == null) throw new IllegalStateException( + "File Cache not initialized on this Node, cannot create Composite Directory without FileCache" + ); + else if (remoteDirectory == null) throw new IllegalStateException( + "Remote Directory must not be null for Composite Directory" + ); + directory = new CompositeDirectory((FSDirectory) localDirectory, (RemoteSegmentStoreDirectory) remoteDirectory, fileCache); } else { directory = directoryFactory.newDirectory(this.indexSettings, path); diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 597a64158be03..76f3bb868d43a 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -101,7 +101,7 @@ public String[] listAll() throws IOException { /** * Removes an existing file in the directory. - * Currently deleting only from local directory as files from remote should not be deleted due to availability reasons + * Currently deleting only from local directory as files from remote should not be deleted as that is taken care by garbage collection logic of remote directory * @param name the name of an existing file. * @throws IOException in case of I/O error */ @@ -181,6 +181,7 @@ public void rename(String source, String dest) throws IOException { localDirectory.rename(source, dest); fileCache.remove(localDirectory.getDirectory().resolve(source)); cacheFile(dest); + fileCache.decRef(localDirectory.getDirectory().resolve(dest)); } /** @@ -243,10 +244,6 @@ public void close() throws IOException { */ public void afterSyncToRemote(Collection files) throws IOException { logger.trace("afterSyncToRemote called for {}", files); - if (remoteDirectory == null) { - logger.trace("afterSyncToRemote called even though remote directory is not set"); - return; - } for (String fileName : files) { /* Decrementing the refCount here for the path so that it becomes eligible for eviction @@ -273,10 +270,9 @@ private String[] getRemoteFiles() throws IOException { remoteFiles = remoteDirectory.listAll(); } catch (NullPointerException e) { /* - There are two scenarios where the listAll() call on remote directory returns NullPointerException: - - When remote directory is not set - - When init() of remote directory has not yet been called - Returning an empty list in the above scenarios + We can encounter NPE when no data has been uploaded to remote store yet and as a result the metadata is empty + Empty metadata means that there are no files currently in remote, hence returning an empty list in this scenario + TODO : Catch the NPE in listAll of RemoteSegmentStoreDirectory itself instead of catching here */ remoteFiles = new String[0]; } @@ -285,7 +281,7 @@ There are two scenarios where the listAll() call on remote directory returns Nul private void cacheFile(String name) throws IOException { Path filePath = localDirectory.getDirectory().resolve(name); - // put will increase the refCount for the path, making sure it is not evicted, wil decrease the ref after it is uploaded to Remote + // put will increase the refCount for the path, making sure it is not evicted, will decrease the ref after it is uploaded to Remote // so that it can be evicted after that // this is just a temporary solution, will pin the file once support for that is added in FileCache // TODO : Pin the above filePath in the file cache once pinning support is added so that it cannot be evicted unless it has been diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java index 7b1163b18727e..f8aed0432cba8 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java @@ -22,8 +22,6 @@ */ @ExperimentalApi public class FullFileCachedIndexInput implements CachedIndexInput { - - private final IndexInput indexInput; private final FileCache fileCache; private final Path path; private final FileCachedIndexInput fileCachedIndexInput; @@ -35,7 +33,6 @@ public class FullFileCachedIndexInput implements CachedIndexInput { public FullFileCachedIndexInput(FileCache fileCache, Path path, IndexInput indexInput) { this.fileCache = fileCache; this.path = path; - this.indexInput = indexInput; fileCachedIndexInput = new FileCachedIndexInput(fileCache, path, indexInput); isClosed = new AtomicBoolean(false); } @@ -54,7 +51,7 @@ public IndexInput getIndexInput() { */ @Override public long length() { - return indexInput.length(); + return fileCachedIndexInput.length(); } /** @@ -71,7 +68,7 @@ public boolean isClosed() { @Override public void close() throws Exception { if (!isClosed.getAndSet(true)) { - indexInput.close(); + fileCachedIndexInput.close(); } } } diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java index cfc6f0277d23d..64649978129c4 100644 --- a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; @@ -32,6 +33,7 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.startsWith; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -50,11 +52,16 @@ public void setup() throws IOException { } public void testListAll() throws IOException { + when(localDirectory.listAll()).thenReturn(new String[]{}); + String[] actualFileNames = compositeDirectory.listAll(); + String[] expectedFileNames = new String[] {}; + assertArrayEquals(expectedFileNames, actualFileNames); + populateMetadata(); when(localDirectory.listAll()).thenReturn(new String[] { "_1.cfe", "_2.cfe", "_0.cfe_block_7", "_0.cfs_block_7" }); - String[] actualFileNames = compositeDirectory.listAll(); - String[] expectedFileNames = new String[] { "_0.cfe", "_0.cfs", "_0.si", "_1.cfe", "_2.cfe", "segments_1" }; + actualFileNames = compositeDirectory.listAll(); + expectedFileNames = new String[] { "_0.cfe", "_0.cfs", "_0.si", "_1.cfe", "_2.cfe", "segments_1" }; assertArrayEquals(expectedFileNames, actualFileNames); } @@ -169,4 +176,14 @@ public void testClose() throws IOException { verify(fileCache).remove(resolvedPath1); verify(fileCache).remove(resolvedPath2); } + + public void testAfterSyncToRemote() throws IOException { + Path basePath = mock(Path.class); + Path resolvedPath = mock(Path.class); + when(basePath.resolve(anyString())).thenReturn(resolvedPath); + when(localDirectory.getDirectory()).thenReturn(basePath); + Collection files = Arrays.asList("_0.si", "_0.cfs"); + compositeDirectory.afterSyncToRemote(files); + verify(fileCache, times(files.size())).decRef(resolvedPath); + } } From de1895e4900a608a4e516260e0575ddbc57235a1 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Thu, 30 May 2024 03:40:33 +0530 Subject: [PATCH 15/22] Handle delete and close in Composite Directory, log current state of FileCache and correct it's clear method and modify unit and integration tests as per review comments Signed-off-by: Shreyansh Ray --- .../remotestore/CompositeDirectoryIT.java | 89 ------ .../remotestore/WritableWarmIT.java | 128 +++++++++ .../org/opensearch/index/IndexService.java | 21 +- .../shard/RemoteStoreRefreshListener.java | 8 +- .../index/store/CompositeDirectory.java | 174 ++++++++---- .../store/remote/filecache/FileCache.java | 11 + .../index/store/remote/utils/FileType.java | 49 ---- .../store/remote/utils/FileTypeUtils.java | 36 +++ .../store/remote/utils/cache/LRUCache.java | 26 +- .../remote/utils/cache/SegmentedCache.java | 13 + .../index/store/CompositeDirectoryTests.java | 258 ++++++++++-------- .../index/store/RemoteDirectoryTests.java | 10 + .../utils/cache/RefCountedCacheTestCase.java | 9 + 13 files changed, 500 insertions(+), 332 deletions(-) delete mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java create mode 100644 server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java delete mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java create mode 100644 server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java deleted file mode 100644 index 3d52e6614f6a3..0000000000000 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/CompositeDirectoryIT.java +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.remotestore; - -import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; - -import org.apache.lucene.store.Directory; -import org.apache.lucene.store.FilterDirectory; -import org.opensearch.action.admin.indices.get.GetIndexRequest; -import org.opensearch.action.admin.indices.get.GetIndexResponse; -import org.opensearch.action.search.SearchResponse; -import org.opensearch.cluster.metadata.IndexMetadata; -import org.opensearch.common.settings.Settings; -import org.opensearch.common.util.FeatureFlags; -import org.opensearch.index.IndexModule; -import org.opensearch.index.IndexService; -import org.opensearch.index.query.QueryBuilders; -import org.opensearch.index.shard.IndexShard; -import org.opensearch.index.store.CompositeDirectory; -import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; -import org.opensearch.indices.IndicesService; -import org.opensearch.test.OpenSearchIntegTestCase; - -import java.util.Map; - -import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; -import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; - -@ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) -@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) -// Uncomment the below line to enable trace level logs for this test for better debugging -// @TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") -public class CompositeDirectoryIT extends RemoteStoreBaseIntegTestCase { - - /* - Disabling MockFSIndexStore plugin as the MockFSDirectoryFactory wraps the FSDirectory over a OpenSearchMockDirectoryWrapper which extends FilterDirectory (whereas FSDirectory extends BaseDirectory) - As a result of this wrapping the local directory of Composite Directory does not satisfy the assertion that local directory must be of type FSDirectory - */ - @Override - protected boolean addMockIndexStorePlugin() { - return false; - } - - @Override - protected Settings featureFlagSettings() { - Settings.Builder featureSettings = Settings.builder(); - featureSettings.put(FeatureFlags.WRITEABLE_REMOTE_INDEX, true); - - return featureSettings.build(); - } - - public void testCompositeDirectory() throws Exception { - Settings settings = Settings.builder() - .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) - .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) - .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), "partial") - .build(); - assertAcked(client().admin().indices().prepareCreate("test-idx-1").setSettings(settings).get()); - - // Check if the Directory initialized for the IndexShard is of Composite Directory type - IndexService indexService = internalCluster().getDataNodeInstance(IndicesService.class).indexService(resolveIndex("test-idx-1")); - IndexShard shard = indexService.getShardOrNull(0); - Directory directory = (((FilterDirectory) (((FilterDirectory) (shard.store().directory())).getDelegate())).getDelegate()); - assertTrue(directory instanceof CompositeDirectory); - - // Verify from the cluster settings if the data locality is partial - GetIndexResponse getIndexResponse = client().admin() - .indices() - .getIndex(new GetIndexRequest().indices("test-idx-1").includeDefaults(true)) - .get(); - Settings indexSettings = getIndexResponse.settings().get("test-idx-1"); - assertEquals("partial", indexSettings.get("index.store.data_locality")); - - // Index data and ensure cluster does not turn red while indexing - Map stats = indexData(10, false, "test-idx-1"); - refresh("test-idx-1"); - ensureGreen("test-idx-1"); - - // Search and verify that the total docs indexed match the search hits - SearchResponse searchResponse3 = client().prepareSearch("test-idx-1").setQuery(QueryBuilders.matchAllQuery()).get(); - assertHitCount(searchResponse3, stats.get(TOTAL_OPERATIONS)); - } -} diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java new file mode 100644 index 0000000000000..690356bca24ce --- /dev/null +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -0,0 +1,128 @@ +/* + * 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.remotestore; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.FilterDirectory; +import org.opensearch.action.admin.indices.get.GetIndexRequest; +import org.opensearch.action.admin.indices.get.GetIndexResponse; +import org.opensearch.action.search.SearchResponse; +import org.opensearch.cluster.metadata.IndexMetadata; +import org.opensearch.common.settings.Settings; +import org.opensearch.common.util.FeatureFlags; +import org.opensearch.index.IndexModule; +import org.opensearch.index.query.QueryBuilders; +import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; +import org.opensearch.index.store.remote.filecache.FileCache; +import org.opensearch.index.store.remote.utils.FileTypeUtils; +import org.opensearch.index.store.remote.utils.cache.CacheUsage; +import org.opensearch.indices.IndicesService; +import org.opensearch.node.Node; +import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.junit.annotations.TestLogging; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertAcked; +import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; + +@ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) +// Uncomment the below line to enable trace level logs for this test for better debugging +@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") +public class WritableWarmIT extends RemoteStoreBaseIntegTestCase { + + protected static final String INDEX_NAME = "test-idx-1"; + protected static final int NUM_DOCS_IN_BULK = 1000; + + /* + Disabling MockFSIndexStore plugin as the MockFSDirectoryFactory wraps the FSDirectory over a OpenSearchMockDirectoryWrapper which extends FilterDirectory (whereas FSDirectory extends BaseDirectory) + As a result of this wrapping the local directory of Composite Directory does not satisfy the assertion that local directory must be of type FSDirectory + */ + @Override + protected boolean addMockIndexStorePlugin() { + return false; + } + + @Override + protected Settings featureFlagSettings() { + Settings.Builder featureSettings = Settings.builder(); + featureSettings.put(FeatureFlags.WRITEABLE_REMOTE_INDEX, true); + + return featureSettings.build(); + } + + public void testWritableWarmBasic() throws Exception { + Settings settings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.PARTIAL.name()) + .build(); + assertAcked(client().admin().indices().prepareCreate(INDEX_NAME).setSettings(settings).get()); + + // Verify from the cluster settings if the data locality is partial + GetIndexResponse getIndexResponse = client().admin() + .indices() + .getIndex(new GetIndexRequest().indices(INDEX_NAME).includeDefaults(true)) + .get(); + Settings indexSettings = getIndexResponse.settings().get(INDEX_NAME); + assertEquals(IndexModule.DataLocalityType.PARTIAL.name(), indexSettings.get(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey())); + + // Ingesting some docs + indexBulk(INDEX_NAME, NUM_DOCS_IN_BULK); + flushAndRefresh(INDEX_NAME); + + // ensuring cluster is green after performing force-merge + ensureGreen(); + + SearchResponse searchResponse = client().prepareSearch(INDEX_NAME).setQuery(QueryBuilders.matchAllQuery()).get(); + // Asserting that search returns same number of docs as ingested + assertHitCount(searchResponse, NUM_DOCS_IN_BULK); + + // Ingesting docs again before force merge + indexBulk(INDEX_NAME, NUM_DOCS_IN_BULK); + flushAndRefresh(INDEX_NAME); + + FileCache fileCache = internalCluster().getDataNodeInstance(Node.class).fileCache(); + IndexShard shard = internalCluster().getDataNodeInstance(IndicesService.class) + .indexService(resolveIndex(INDEX_NAME)) + .getShardOrNull(0); + Directory directory = (((FilterDirectory) (((FilterDirectory) (shard.store().directory())).getDelegate())).getDelegate()); + + // Force merging the index + Set filesBeforeMerge = new HashSet<>(Arrays.asList(directory.listAll())); + client().admin().indices().prepareForceMerge(INDEX_NAME).setMaxNumSegments(1).get(); + flushAndRefresh(INDEX_NAME); + Set filesAfterMerge = new HashSet<>(Arrays.asList(directory.listAll())); + + CacheUsage usageBeforePrune = fileCache.usage(); + fileCache.prune(); + CacheUsage usageAfterPrune = fileCache.usage(); + + Set filesFromPreviousGenStillPresent = filesBeforeMerge.stream() + .filter(filesAfterMerge::contains) + .filter(file -> !FileTypeUtils.isLockFile(file)) + .collect(Collectors.toUnmodifiableSet()); + + // Asserting that after merge all the files from previous gen are no more part of the directory + assertTrue(filesFromPreviousGenStillPresent.isEmpty()); + // Asserting that after the merge, refCount of some files in FileCache dropped to zero which resulted in their eviction after + // pruning + assertTrue(usageAfterPrune.usage() < usageBeforePrune.usage()); + + // Clearing the file cache to avoid any file leaks + fileCache.clear(); + } +} diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index d021b290e585b..6cceae5ce7e9f 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -39,7 +39,6 @@ import org.apache.lucene.search.Sort; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; -import org.apache.lucene.store.FSDirectory; import org.apache.lucene.util.Accountable; import org.opensearch.client.Client; import org.opensearch.cluster.metadata.IndexMetadata; @@ -94,7 +93,6 @@ import org.opensearch.index.shard.ShardPath; import org.opensearch.index.similarity.SimilarityService; import org.opensearch.index.store.CompositeDirectory; -import org.opensearch.index.store.RemoteSegmentStoreDirectory; import org.opensearch.index.store.RemoteSegmentStoreDirectoryFactory; import org.opensearch.index.store.Store; import org.opensearch.index.store.remote.filecache.FileCache; @@ -617,25 +615,8 @@ public synchronized IndexShard createShard( if (FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING) && // TODO : Need to remove this check after support for hot indices is added in Composite Directory this.indexSettings.isStoreLocalityPartial()) { - /* - Currently Composite Directory only supports local directory to be of type FSDirectory - The reason is that FileCache currently has it key type as Path - Composite Directory currently uses FSDirectory's getDirectory() method to fetch and use the Path for operating on FileCache - TODO : Refactor FileCache to have key in form of String instead of Path. Once that is done we can remove this assertion - */ Directory localDirectory = directoryFactory.newDirectory(this.indexSettings, path); - - if (localDirectory instanceof FSDirectory == false) throw new IllegalStateException( - "For Composite Directory, local directory must be of type FSDirectory" - ); - else if (fileCache == null) throw new IllegalStateException( - "File Cache not initialized on this Node, cannot create Composite Directory without FileCache" - ); - else if (remoteDirectory == null) throw new IllegalStateException( - "Remote Directory must not be null for Composite Directory" - ); - - directory = new CompositeDirectory((FSDirectory) localDirectory, (RemoteSegmentStoreDirectory) remoteDirectory, fileCache); + directory = new CompositeDirectory(localDirectory, remoteDirectory, fileCache); } else { directory = directoryFactory.newDirectory(this.indexSettings, path); } 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 e9f996e24f679..601fb56876039 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -266,7 +266,6 @@ private boolean syncSegments() { Collection segmentsToRefresh = localSegmentsPostRefresh.stream() .filter(file -> !skipUpload(file)) .collect(Collectors.toList()); - Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); CountDownLatch latch = new CountDownLatch(1); ActionListener segmentUploadsCompletedListener = new LatchedActionListener<>(new ActionListener<>() { @@ -289,9 +288,6 @@ public void onResponse(Void unused) { // At this point since we have uploaded new segments, segment infos and segment metadata file, // along with marking minSeqNoToKeep, upload has succeeded completely. successful.set(true); - if (directory instanceof CompositeDirectory) { - ((CompositeDirectory) directory).afterSyncToRemote(segmentsToRefresh); - } } catch (Exception e) { // We don't want to fail refresh if upload of new segments fails. The missed segments will be re-tried // as part of exponential back-off retry logic. This should not affect durability of the indexed data @@ -448,6 +444,7 @@ private void uploadNewSegments( logger.debug("Effective new segments files to upload {}", filteredFiles); ActionListener> mappedListener = ActionListener.map(listener, resp -> null); GroupedActionListener batchUploadListener = new GroupedActionListener<>(mappedListener, filteredFiles.size()); + Directory directory = ((FilterDirectory) (((FilterDirectory) storeDirectory).getDelegate())).getDelegate(); for (String src : filteredFiles) { // Initializing listener here to ensure that the stats increment operations are thread-safe @@ -455,6 +452,9 @@ private void uploadNewSegments( ActionListener aggregatedListener = ActionListener.wrap(resp -> { statsListener.onSuccess(src); batchUploadListener.onResponse(resp); + if (directory instanceof CompositeDirectory) { + ((CompositeDirectory) directory).afterSyncToRemote(src); + } }, ex -> { logger.warn(() -> new ParameterizedMessage("Exception: [{}] while uploading segment files", ex), ex); if (ex instanceof CorruptIndexException) { diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 76f3bb868d43a..ecc6a95475170 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -10,6 +10,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.Directory; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.FilterDirectory; import org.apache.lucene.store.IOContext; @@ -23,9 +24,8 @@ import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.filecache.FullFileCachedIndexInput; -import org.opensearch.index.store.remote.metadata.RemoteSegmentMetadata; import org.opensearch.index.store.remote.utils.BlockIOContext; -import org.opensearch.index.store.remote.utils.FileType; +import org.opensearch.index.store.remote.utils.FileTypeUtils; import org.opensearch.index.store.remote.utils.TransferManager; import java.io.FileNotFoundException; @@ -53,6 +53,7 @@ public class CompositeDirectory extends FilterDirectory { private final RemoteSegmentStoreDirectory remoteDirectory; private final FileCache fileCache; private final TransferManager transferManager; + private final Set pendingDeletes; /** * Constructor to initialise the composite directory @@ -60,11 +61,13 @@ public class CompositeDirectory extends FilterDirectory { * @param remoteDirectory corresponding to the remote directory * @param fileCache used to cache the remote files locally */ - public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirectory remoteDirectory, FileCache fileCache) { + public CompositeDirectory(Directory localDirectory, Directory remoteDirectory, FileCache fileCache) { super(localDirectory); - this.localDirectory = localDirectory; - this.remoteDirectory = remoteDirectory; + validate(localDirectory, remoteDirectory, fileCache); + this.localDirectory = (FSDirectory) localDirectory; + this.remoteDirectory = (RemoteSegmentStoreDirectory) remoteDirectory; this.fileCache = fileCache; + this.pendingDeletes = new HashSet<>(); transferManager = new TransferManager( (name, position, length) -> new InputStreamIndexInput( remoteDirectory.openInput(name, new BlockIOContext(IOContext.DEFAULT, position, length)), @@ -76,26 +79,28 @@ public CompositeDirectory(FSDirectory localDirectory, RemoteSegmentStoreDirector /** * Returns names of all files stored in this directory in sorted order - * Does not include locally stored block files (having _block_ in their names) + * Does not include locally stored block files (having _block_ in their names) and files pending deletion * * @throws IOException in case of I/O error */ @Override public String[] listAll() throws IOException { - logger.trace("listAll() called"); + ensureOpen(); + logger.trace("Composite Directory[{}]: listAll() called", this::toString); String[] localFiles = localDirectory.listAll(); - logger.trace("Local Directory files : {}", () -> Arrays.toString(localFiles)); Set allFiles = new HashSet<>(Arrays.asList(localFiles)); String[] remoteFiles = getRemoteFiles(); allFiles.addAll(Arrays.asList(remoteFiles)); - logger.trace("Remote Directory files : {}", () -> Arrays.toString(remoteFiles)); - Set localLuceneFiles = allFiles.stream() - .filter(file -> !FileType.isBlockFile(file)) + logger.trace("Composite Directory[{}]: Local Directory files - {}", this::toString, () -> Arrays.toString(localFiles)); + logger.trace("Composite Directory[{}]: Remote Directory files - {}", this::toString, () -> Arrays.toString(remoteFiles)); + Set nonBlockLuceneFiles = allFiles.stream() + .filter(file -> !FileTypeUtils.isBlockFile(file)) + .filter(file -> !pendingDeletes.contains(file)) .collect(Collectors.toUnmodifiableSet()); - String[] files = new String[localLuceneFiles.size()]; - localLuceneFiles.toArray(files); + String[] files = new String[nonBlockLuceneFiles.size()]; + nonBlockLuceneFiles.toArray(files); Arrays.sort(files); - logger.trace("listAll() returns : {}", () -> Arrays.toString(files)); + logger.trace("Composite Directory[{}]: listAll() returns : {}", this::toString, () -> Arrays.toString(files)); return files; } @@ -107,15 +112,15 @@ public String[] listAll() throws IOException { */ @Override public void deleteFile(String name) throws IOException { - logger.trace("deleteFile() called {}", name); - if (isTempFile(name)) { + ensureOpen(); + ensureFileNotDeleted(name); + logger.trace("Composite Directory[{}]: deleteFile() called {}", this::toString, () -> name); + if (FileTypeUtils.isTempFile(name)) { localDirectory.deleteFile(name); + } else if (Arrays.asList(listAll()).contains(name) == false) { + throw new NoSuchFileException("File " + name + " not found in directory"); } else { - /* - Not deleting from localDirectory directly since it causes a race condition when the localDirectory deletes a file, and it ends up in pendingDeletion state. - Meanwhile, fileCache on removal deletes the file directly via the Files class and later when the directory tries to delete the files pending for deletion (which happens before creating a new file), it causes NoSuchFileException and new file creation fails - */ - fileCache.remove(localDirectory.getDirectory().resolve(name)); + pendingDeletes.add(name); } } @@ -127,19 +132,31 @@ Meanwhile, fileCache on removal deletes the file directly via the Files class an */ @Override public long fileLength(String name) throws IOException { - logger.trace("fileLength() called {}", name); + ensureOpen(); + ensureFileNotDeleted(name); + logger.trace("Composite Directory[{}]: fileLength() called {}", this::toString, () -> name); long fileLength; Path key = localDirectory.getDirectory().resolve(name); - if (isTempFile(name) || fileCache.get(key) != null) { + if (FileTypeUtils.isTempFile(name) || fileCache.get(key) != null) { try { fileLength = localDirectory.fileLength(name); - logger.trace("fileLength from Local {}", fileLength); + logger.trace( + "Composite Directory[{}]: fileLength of {} fetched from Local - {}", + this::toString, + () -> name, + () -> fileLength + ); } finally { fileCache.decRef(key); } } else { fileLength = remoteDirectory.fileLength(name); - logger.trace("fileLength from Remote {}", fileLength); + logger.trace( + "Composite Directory[{}]: fileLength of {} fetched from Remote - {}", + this::toString, + () -> name, + () -> fileLength + ); } return fileLength; } @@ -152,7 +169,12 @@ public long fileLength(String name) throws IOException { */ @Override public IndexOutput createOutput(String name, IOContext context) throws IOException { - logger.trace("createOutput() called {}", name); + ensureOpen(); + // If file was deleted earlier, removing it from the deleted list + if (pendingDeletes.contains(name)) { + pendingDeletes.remove(name); + } + logger.trace("Composite Directory[{}]: createOutput() called {}", this::toString, () -> name); // The CloseableFilterIndexOutput will ensure that the file is added to FileCache once write is completed on this file return new CloseableFilterIndexOutput(localDirectory.createOutput(name, context), name, this::cacheFile); } @@ -163,10 +185,11 @@ public IndexOutput createOutput(String name, IOContext context) throws IOExcepti */ @Override public void sync(Collection names) throws IOException { - logger.trace("sync() called {}", names); + ensureOpen(); + logger.trace("Composite Directory[{}]: sync() called {}", this::toString, () -> names); Collection remoteFiles = Arrays.asList(getRemoteFiles()); Collection filesToSync = names.stream().filter(name -> remoteFiles.contains(name) == false).collect(Collectors.toList()); - logger.trace("Synced files : {}", filesToSync); + logger.trace("Composite Directory[{}]: Synced files : {}", this::toString, () -> filesToSync); localDirectory.sync(filesToSync); } @@ -177,11 +200,12 @@ public void sync(Collection names) throws IOException { */ @Override public void rename(String source, String dest) throws IOException { - logger.trace("rename() called {}, {}", source, dest); + ensureOpen(); + ensureFileNotDeleted(source); + logger.trace("Composite Directory[{}]: rename() called : source-{}, dest-{}", this::toString, () -> source, () -> dest); localDirectory.rename(source, dest); fileCache.remove(localDirectory.getDirectory().resolve(source)); cacheFile(dest); - fileCache.decRef(localDirectory.getDirectory().resolve(dest)); } /** @@ -192,18 +216,20 @@ public void rename(String source, String dest) throws IOException { */ @Override public IndexInput openInput(String name, IOContext context) throws IOException { - logger.trace("openInput() called {}", name); + ensureOpen(); + ensureFileNotDeleted(name); + logger.trace("Composite Directory[{}]: openInput() called {}", this::toString, () -> name); // We aren't tracking temporary files (created via createTempOutput) currently in FileCache as these are created and then deleted // within a very short span of time // We will be reading them directory from the local directory - if (isTempFile(name)) { + if (FileTypeUtils.isTempFile(name)) { return localDirectory.openInput(name, context); } // Return directly from the FileCache (via TransferManager) if complete file is present Path key = localDirectory.getDirectory().resolve(name); CachedIndexInput indexInput = fileCache.get(key); if (indexInput != null) { - logger.trace("Complete file found in FileCache"); + logger.trace("Composite Directory[{}]: Complete file {} found in FileCache", this::toString, () -> name); try { return indexInput.getIndexInput().clone(); } finally { @@ -212,9 +238,16 @@ public IndexInput openInput(String name, IOContext context) throws IOException { } // If file has been uploaded to the Remote Store, fetch it from the Remote Store in blocks via OnDemandCompositeBlockIndexInput else { - logger.trace("Complete file not in FileCache, to be fetched in Blocks from Remote"); - RemoteSegmentMetadata remoteSegmentMetadata = remoteDirectory.readLatestMetadataFile(); - RemoteSegmentStoreDirectory.UploadedSegmentMetadata uploadedSegmentMetadata = remoteSegmentMetadata.getMetadata().get(name); + logger.trace( + "Composite Directory[{}]: Complete file {} not in FileCache, to be fetched in Blocks from Remote", + this::toString, + () -> name + ); + RemoteSegmentStoreDirectory.UploadedSegmentMetadata uploadedSegmentMetadata = remoteDirectory.getSegmentsUploadedToRemoteStore() + .get(name); + if (uploadedSegmentMetadata == null) { + throw new NoSuchFileException("File " + name + " not found in directory"); + } // TODO : Refactor FileInfo and OnDemandBlockSnapshotIndexInput to more generic names as they are not Remote Snapshot specific BlobStoreIndexShardSnapshot.FileInfo fileInfo = new BlobStoreIndexShardSnapshot.FileInfo( name, @@ -226,39 +259,68 @@ public IndexInput openInput(String name, IOContext context) throws IOException { } /** - * Closes the directory + * Closing the local directory here * @throws IOException in case of I/O error */ @Override public void close() throws IOException { - Arrays.stream(localDirectory.listAll()).forEach(f -> fileCache.remove(localDirectory.getDirectory().resolve(f))); + ensureOpen(); + logger.trace("Composite Directory[{}]: close() called", this::toString); localDirectory.close(); - remoteDirectory.close(); } /** * Function to perform operations once files have been uploaded to Remote Store * Currently deleting the local files here, as once uploaded to Remote, local files become eligible for eviction from FileCache - * @param files : recent files which have been successfully uploaded to Remote Store - * @throws IOException in case of I/O error + * @param file : recent files which have been successfully uploaded to Remote Store */ - public void afterSyncToRemote(Collection files) throws IOException { - logger.trace("afterSyncToRemote called for {}", files); - for (String fileName : files) { - /* - Decrementing the refCount here for the path so that it becomes eligible for eviction - This is a temporary solution until pinning support is added - TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache - Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality - */ - logger.trace("File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", fileName); - fileCache.decRef(localDirectory.getDirectory().resolve(fileName)); - // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); - } + public void afterSyncToRemote(String file) { + ensureOpen(); + /* + Decrementing the refCount here for the path so that it becomes eligible for eviction + This is a temporary solution until pinning support is added + TODO - Unpin the files here from FileCache so that they become eligible for eviction, once pinning/unpinning support is added in FileCache + Uncomment the below commented line(to remove the file from cache once uploaded) to test block based functionality + */ + logger.trace( + "Composite Directory[{}]: File {} uploaded to Remote Store and now can be eligible for eviction in FileCache", + this::toString, + () -> file + ); + fileCache.decRef(localDirectory.getDirectory().resolve(file)); + // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); + } + + /** + * Basic validations for Composite Directory parameters (null checks and instance type checks) + * + * Note: Currently Composite Directory only supports local directory to be of type FSDirectory + * The reason is that FileCache currently has it key type as Path + * Composite Directory currently uses FSDirectory's getDirectory() method to fetch and use the Path for operating on FileCache + * TODO : Refactor FileCache to have key in form of String instead of Path. Once that is done we can remove this assertion + */ + private void validate(Directory localDirectory, Directory remoteDirectory, FileCache fileCache) { + if (localDirectory == null || remoteDirectory == null) throw new IllegalStateException( + "Local and remote directory cannot be null for Composite Directory" + ); + if (fileCache == null) throw new IllegalStateException( + "File Cache not initialized on this Node, cannot create Composite Directory without FileCache" + ); + if (localDirectory instanceof FSDirectory == false) throw new IllegalStateException( + "For Composite Directory, local directory must be of type FSDirectory" + ); + if (remoteDirectory instanceof RemoteSegmentStoreDirectory == false) throw new IllegalStateException( + "For Composite Directory, remote directory must be of type RemoteSegmentStoreDirectory" + ); } - private boolean isTempFile(String name) { - return name.endsWith(".tmp"); + /** + * Ensure that the file has not already been deleted + */ + private void ensureFileNotDeleted(String name) throws IOException { + if (pendingDeletes.contains(name)) { + throw new NoSuchFileException("File " + name + " is already pending delete"); + } } /** diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java index e61e5ecd4084a..a8aa33a977cb8 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCache.java @@ -8,6 +8,8 @@ package org.opensearch.index.store.remote.filecache; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.store.IndexInput; import org.opensearch.common.annotation.PublicApi; import org.opensearch.core.common.breaker.CircuitBreaker; @@ -47,6 +49,7 @@ */ @PublicApi(since = "2.7.0") public class FileCache implements RefCountedCache { + private static final Logger logger = LogManager.getLogger(FileCache.class); private final SegmentedCache theCache; private final CircuitBreaker circuitBreaker; @@ -139,6 +142,14 @@ public CacheStats stats() { return theCache.stats(); } + // To be used only for debugging purposes + public void logCurrentState() { + logger.trace("CURRENT STATE OF FILE CACHE \n"); + CacheUsage cacheUsage = theCache.usage(); + logger.trace("Total Usage: " + cacheUsage.usage() + " , Active Usage: " + cacheUsage.activeUsage()); + theCache.logCurrentState(); + } + /** * Ensures that the PARENT breaker is not tripped when an entry is added to the cache * @param filePath the path key for which entry is added diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java b/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java deleted file mode 100644 index e340c82ba9ba3..0000000000000 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/FileType.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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.store.remote.utils; - -import org.opensearch.common.annotation.ExperimentalApi; - -/** - * Enum to represent whether a file is block or not - * - * @opensearch.experimental - */ -@ExperimentalApi -public enum FileType { - /** - * Block file - */ - BLOCK(".*_block_.*"), - /** - * Full file - Non-Block - */ - FULL(".*"); - - private final String pattern; - - FileType(String pattern) { - this.pattern = pattern; - } - - /** - * Returns if the fileType is a block file or not - */ - public static boolean isBlockFile(FileType fileType) { - return fileType.equals(FileType.BLOCK); - } - - /** - * Returns if the fileName is block file or not - */ - public static boolean isBlockFile(String fileName) { - if (fileName.matches(FileType.BLOCK.pattern)) return true; - return false; - } -} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java b/server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java new file mode 100644 index 0000000000000..c20533e0e5ff1 --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java @@ -0,0 +1,36 @@ +/* + * 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.store.remote.utils; + +import org.opensearch.common.annotation.ExperimentalApi; + +/** + * Utility class for checking file types + * + * @opensearch.experimental + */ +@ExperimentalApi +public class FileTypeUtils { + + public static boolean isTempFile(String name) { + return name.endsWith(".tmp"); + } + + public static boolean isBlockFile(String name) { + return name.contains("_block_"); + } + + public static boolean isExtraFSFile(String name) { + return name.startsWith("extra"); + } + + public static boolean isLockFile(String name) { + return name.endsWith(".lock"); + } +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/cache/LRUCache.java b/server/src/main/java/org/opensearch/index/store/remote/utils/cache/LRUCache.java index 03d03711f914a..7f7d42e8fbce8 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/cache/LRUCache.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/cache/LRUCache.java @@ -8,6 +8,8 @@ package org.opensearch.index.store.remote.utils.cache; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.RemovalReason; @@ -19,6 +21,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; +import java.util.Map; import java.util.Objects; import java.util.concurrent.locks.ReentrantLock; import java.util.function.BiFunction; @@ -43,6 +46,7 @@ * @opensearch.internal */ class LRUCache implements RefCountedCache { + private static final Logger logger = LogManager.getLogger(LRUCache.class); private final long capacity; private final HashMap> data; @@ -192,8 +196,10 @@ public void clear() { usage = 0L; activeUsage = 0L; lru.clear(); - for (Node node : data.values()) { - data.remove(node.key); + final Iterator> iterator = data.values().iterator(); + while (iterator.hasNext()) { + Node node = iterator.next(); + iterator.remove(); statsCounter.recordRemoval(node.weight); listener.onRemoval(new RemovalNotification<>(node.key, node.value, RemovalReason.EXPLICIT)); } @@ -300,6 +306,22 @@ public CacheStats stats() { } } + // To be used only for debugging purposes + public void logCurrentState() { + lock.lock(); + try { + String allFiles = "\n"; + for (Map.Entry> entry : data.entrySet()) { + String path = entry.getKey().toString(); + String file = path.substring(path.lastIndexOf('/')); + allFiles += file + " [RefCount: " + entry.getValue().refCount + " , Weight: " + entry.getValue().weight + " ]\n"; + } + logger.trace("Cache entries : " + allFiles); + } finally { + lock.unlock(); + } + } + private void addNode(K key, V value) { final long weight = weigher.weightOf(value); Node newNode = new Node<>(key, value, weight); diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/cache/SegmentedCache.java b/server/src/main/java/org/opensearch/index/store/remote/utils/cache/SegmentedCache.java index d3eb03df37e1b..2ea7ea8dbee12 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/cache/SegmentedCache.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/cache/SegmentedCache.java @@ -8,6 +8,8 @@ package org.opensearch.index.store.remote.utils.cache; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.opensearch.common.cache.RemovalListener; import org.opensearch.common.cache.RemovalNotification; import org.opensearch.common.cache.Weigher; @@ -25,6 +27,7 @@ * @opensearch.internal */ public class SegmentedCache implements RefCountedCache { + private static final Logger logger = LogManager.getLogger(SegmentedCache.class); private static final int HASH_BITS = 0x7fffffff; @@ -183,6 +186,16 @@ public CacheStats stats() { return new CacheStats(hitCount, missCount, removeCount, removeWeight, replaceCount, evictionCount, evictionWeight); } + // To be used only for debugging purposes + public void logCurrentState() { + int i = 0; + for (RefCountedCache cache : table) { + logger.trace("SegmentedCache " + i); + ((LRUCache) cache).logCurrentState(); + i++; + } + } + enum SingletonWeigher implements Weigher { INSTANCE; diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java index 64649978129c4..2045172f623bf 100644 --- a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -10,32 +10,28 @@ import com.carrotsearch.randomizedtesting.annotations.ThreadLeakFilters; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.FSDirectory; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.IndexOutput; +import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.common.breaker.NoopCircuitBreaker; import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; -import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.filecache.FullFileCachedIndexInput; +import org.opensearch.index.store.remote.filecache.FileCacheFactory; +import org.opensearch.index.store.remote.filecache.FileCachedIndexInput; +import org.opensearch.index.store.remote.utils.FileTypeUtils; import org.junit.Before; import java.io.IOException; +import java.nio.file.NoSuchFileException; import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.List; -import java.util.Map; - -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.startsWith; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) public class CompositeDirectoryTests extends BaseRemoteSegmentStoreDirectoryTests { @@ -43,147 +39,185 @@ public class CompositeDirectoryTests extends BaseRemoteSegmentStoreDirectoryTest private FSDirectory localDirectory; private CompositeDirectory compositeDirectory; + private final static String[] LOCAL_FILES = new String[] { "_1.cfe", "_2.cfe", "_0.cfe_block_7", "_0.cfs_block_7", "temp_file.tmp" }; + private final static String FILE_PRESENT_LOCALLY = "_1.cfe"; + private final static String FILE_PRESENT_IN_REMOTE_ONLY = "_0.si"; + private final static String NON_EXISTENT_FILE = "non_existent_file"; + private final static String NEW_FILE = "new_file"; + private final static String TEMP_FILE = "temp_file.tmp"; + private final static int FILE_CACHE_CAPACITY = 10000; + @Before public void setup() throws IOException { setupRemoteSegmentStoreDirectory(); - localDirectory = mock(FSDirectory.class); - fileCache = mock(FileCache.class); + populateMetadata(); + remoteSegmentStoreDirectory.init(); + localDirectory = FSDirectory.open(createTempDir()); + removeExtraFSFiles(); + fileCache = FileCacheFactory.createConcurrentLRUFileCache(FILE_CACHE_CAPACITY, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); compositeDirectory = new CompositeDirectory(localDirectory, remoteSegmentStoreDirectory, fileCache); + addFilesToDirectory(LOCAL_FILES); } public void testListAll() throws IOException { - when(localDirectory.listAll()).thenReturn(new String[]{}); String[] actualFileNames = compositeDirectory.listAll(); - String[] expectedFileNames = new String[] {}; - assertArrayEquals(expectedFileNames, actualFileNames); - - populateMetadata(); - when(localDirectory.listAll()).thenReturn(new String[] { "_1.cfe", "_2.cfe", "_0.cfe_block_7", "_0.cfs_block_7" }); - - actualFileNames = compositeDirectory.listAll(); - expectedFileNames = new String[] { "_0.cfe", "_0.cfs", "_0.si", "_1.cfe", "_2.cfe", "segments_1" }; + String[] expectedFileNames = new String[] { "_0.cfe", "_0.cfs", "_0.si", "_1.cfe", "_2.cfe", "segments_1", "temp_file.tmp" }; assertArrayEquals(expectedFileNames, actualFileNames); } public void testDeleteFile() throws IOException { - Path basePath = mock(Path.class); - Path resolvedPath = mock(Path.class); - when(basePath.resolve(anyString())).thenReturn(resolvedPath); - when(localDirectory.getDirectory()).thenReturn(basePath); - - compositeDirectory.deleteFile("_0.tmp"); - verify(localDirectory).deleteFile("_0.tmp"); - - compositeDirectory.deleteFile("_0.si"); - verify(fileCache).remove(resolvedPath); + assertTrue(existsInCompositeDirectory(FILE_PRESENT_LOCALLY)); + // Read existing file from directory + IndexInput indexInput = compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT); + // Delete the file and assert that it no more is a part of the directory + compositeDirectory.deleteFile(FILE_PRESENT_LOCALLY); + assertFalse(existsInCompositeDirectory(FILE_PRESENT_LOCALLY)); + // On trying any operations(such as openInput) on the deleted file again, it should throw NoSuchFileException + assertThrows( + "File _1.cfe is already pending delete", + NoSuchFileException.class, + () -> compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT) + ); + // Trying to delete it again should also result in NoSuchFileException + assertThrows( + "File" + FILE_PRESENT_LOCALLY + " is already pending delete", + NoSuchFileException.class, + () -> compositeDirectory.deleteFile("_1.cfe") + ); + // Even though the file is deleted, IndexInputs opened prior to deletion should work fine until closed + indexInput.getFilePointer(); + indexInput.close(); + assertThrows(RuntimeException.class, indexInput::getFilePointer); + // Files which do not exist in directory should throw NoSuchFileException + assertFalse(existsInCompositeDirectory(NON_EXISTENT_FILE)); + assertThrows( + "File " + NON_EXISTENT_FILE + " not found in directory", + NoSuchFileException.class, + () -> compositeDirectory.deleteFile(NON_EXISTENT_FILE) + ); } public void testFileLength() throws IOException { - populateMetadata(); - remoteSegmentStoreDirectory.init(); - Path basePath = mock(Path.class); - Path resolvedPath = mock(Path.class); - when(basePath.resolve("_0.si")).thenReturn(resolvedPath); - when(localDirectory.getDirectory()).thenReturn(basePath); - when(localDirectory.fileLength("_0.si")).thenReturn(7L); - // File present locally - CachedIndexInput indexInput = mock(CachedIndexInput.class); - when(fileCache.get(resolvedPath)).thenReturn(indexInput); - assertEquals(compositeDirectory.fileLength("_0.si"), 7L); - verify(localDirectory).fileLength(startsWith("_0.si")); - - // File not present locally - Map uploadedSegments = remoteSegmentStoreDirectory - .getSegmentsUploadedToRemoteStore(); - assertTrue(uploadedSegments.containsKey("_0.si")); - when(fileCache.get(resolvedPath)).thenReturn(null); - assertEquals(compositeDirectory.fileLength("_0.si"), uploadedSegments.get("_0.si").getLength()); + assertTrue(existsInLocalDirectory(FILE_PRESENT_LOCALLY)); + assertFalse(existsInRemoteDirectory(FILE_PRESENT_LOCALLY)); + assertEquals(compositeDirectory.fileLength(FILE_PRESENT_LOCALLY), localDirectory.fileLength(FILE_PRESENT_LOCALLY)); + + // File not present locally - present in Remote + assertFalse(existsInLocalDirectory(FILE_PRESENT_IN_REMOTE_ONLY)); + assertTrue(existsInRemoteDirectory(FILE_PRESENT_IN_REMOTE_ONLY)); + assertEquals( + compositeDirectory.fileLength(FILE_PRESENT_IN_REMOTE_ONLY), + remoteSegmentStoreDirectory.fileLength(FILE_PRESENT_IN_REMOTE_ONLY) + ); + + // File not present in both local and remote + assertFalse(Arrays.asList(compositeDirectory.listAll()).contains(NON_EXISTENT_FILE)); + assertThrows(NoSuchFileException.class, () -> compositeDirectory.fileLength(NON_EXISTENT_FILE)); } public void testCreateOutput() throws IOException { - IndexOutput indexOutput = mock(IndexOutput.class); - when(localDirectory.createOutput("_0.si", IOContext.DEFAULT)).thenReturn(indexOutput); - IndexOutput actualIndexOutput = compositeDirectory.createOutput("_0.si", IOContext.DEFAULT); - assert actualIndexOutput instanceof CloseableFilterIndexOutput; - verify(localDirectory).createOutput("_0.si", IOContext.DEFAULT); + try (IndexOutput indexOutput = compositeDirectory.createOutput(NEW_FILE, IOContext.DEFAULT)) { + // File not present in FileCache until the indexOutput is Closed + assertNull(fileCache.get(localDirectory.getDirectory().resolve(NEW_FILE))); + } + // File present in FileCache after the indexOutput is Closed + assertNotNull(fileCache.get(localDirectory.getDirectory().resolve(NEW_FILE))); } public void testSync() throws IOException { - populateMetadata(); - remoteSegmentStoreDirectory.init(); - Collection names = List.of("_0.cfe", "_0.cfs", "_1.cfe", "_1.cfs", "_2.nvm", "segments_1"); + // All the files in the below list are present either locally or on remote, so sync should work as expected + Collection names = List.of("_0.cfe", "_0.cfs", "_0.si", "_1.cfe", "_2.cfe", "segments_1"); compositeDirectory.sync(names); - verify(localDirectory).sync(List.of("_1.cfe", "_1.cfs", "_2.nvm")); + // Below list contains a non-existent file, hence will throw an error + Collection names1 = List.of("_0.cfe", "_0.cfs", "_0.si", "_1.cfe", "_2.cfe", "segments_1", "non_existent_file"); + assertThrows(NoSuchFileException.class, () -> compositeDirectory.sync(names1)); } public void testRename() throws IOException { - Path basePath = mock(Path.class); - Path resolvedPathOldFile = mock(Path.class); - Path resolvedPathNewFile = mock(Path.class); - when(basePath.resolve("old_file_name")).thenReturn(resolvedPathOldFile); - when(basePath.resolve("new_file_name")).thenReturn(resolvedPathNewFile); - when(localDirectory.getDirectory()).thenReturn(basePath); - CachedIndexInput indexInput = mock(CachedIndexInput.class); - when(fileCache.get(resolvedPathNewFile)).thenReturn(indexInput); - compositeDirectory.rename("old_file_name", "new_file_name"); - verify(localDirectory).rename("old_file_name", "new_file_name"); - verify(fileCache).remove(resolvedPathOldFile); - verify(fileCache).put(eq(resolvedPathNewFile), any(FullFileCachedIndexInput.class)); + // Rename should work as expected for file present in directory + assertTrue(existsInCompositeDirectory(FILE_PRESENT_LOCALLY)); + compositeDirectory.rename(FILE_PRESENT_LOCALLY, "_1_new.cfe"); + // Should throw error for file not present + assertThrows(NoSuchFileException.class, () -> compositeDirectory.rename(NON_EXISTENT_FILE, "_1_new.cfe")); } public void testOpenInput() throws IOException { - populateMetadata(); - remoteSegmentStoreDirectory.init(); - Path basePath = mock(Path.class); - Path resolvedPathInCache = mock(Path.class); - Path resolvedPathNotInCache = mock(Path.class); - when(basePath.resolve("_0.si")).thenReturn(resolvedPathInCache); - when(basePath.resolve("_0.cfs")).thenReturn(resolvedPathNotInCache); - when(localDirectory.getDirectory()).thenReturn(basePath); - CachedIndexInput cachedIndexInput = mock(CachedIndexInput.class); - IndexInput localIndexInput = mock(IndexInput.class); - IndexInput indexInput = mock(IndexInput.class); - when(fileCache.get(resolvedPathInCache)).thenReturn(cachedIndexInput); - when(fileCache.compute(eq(resolvedPathInCache), any())).thenReturn(cachedIndexInput); - when(cachedIndexInput.getIndexInput()).thenReturn(indexInput); - when(indexInput.clone()).thenReturn(indexInput); - when(fileCache.get(resolvedPathNotInCache)).thenReturn(null); + // File not present in Directory + assertFalse(existsInCompositeDirectory(NON_EXISTENT_FILE)); + assertThrows(NoSuchFileException.class, () -> compositeDirectory.openInput(NON_EXISTENT_FILE, IOContext.DEFAULT)); // Temp file, read directly form local directory - when(localDirectory.openInput("_0.tmp", IOContext.DEFAULT)).thenReturn(localIndexInput); - assertEquals(compositeDirectory.openInput("_0.tmp", IOContext.DEFAULT), localIndexInput); - verify(localDirectory).openInput("_0.tmp", IOContext.DEFAULT); + assertTrue(existsInLocalDirectory(TEMP_FILE) && FileTypeUtils.isTempFile(TEMP_FILE)); + assertEquals( + compositeDirectory.openInput(TEMP_FILE, IOContext.DEFAULT).toString(), + localDirectory.openInput(TEMP_FILE, IOContext.DEFAULT).toString() + ); // File present in file cache - assertEquals(compositeDirectory.openInput("_0.si", IOContext.DEFAULT), indexInput); + assertNotNull(fileCache.get(getFilePath(FILE_PRESENT_LOCALLY))); + assertTrue(compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT) instanceof FileCachedIndexInput); // File present in Remote - IndexInput indexInput1 = compositeDirectory.openInput("_0.cfs", IOContext.DEFAULT); - assert indexInput1 instanceof OnDemandBlockSnapshotIndexInput; + assertFalse(existsInLocalDirectory(FILE_PRESENT_IN_REMOTE_ONLY)); + assertTrue(existsInRemoteDirectory(FILE_PRESENT_IN_REMOTE_ONLY)); + assertTrue(compositeDirectory.openInput(FILE_PRESENT_IN_REMOTE_ONLY, IOContext.DEFAULT) instanceof OnDemandBlockSnapshotIndexInput); } public void testClose() throws IOException { - Path basePath = mock(Path.class); - Path resolvedPath1 = mock(Path.class); - Path resolvedPath2 = mock(Path.class); - when(basePath.resolve("_0.si")).thenReturn(resolvedPath1); - when(basePath.resolve("_0.cfs")).thenReturn(resolvedPath2); - when(localDirectory.getDirectory()).thenReturn(basePath); - when(localDirectory.listAll()).thenReturn(new String[] { "_0.si", "_0.cfs" }); + // Similar to delete, when close is called existing openInput should be able to function properly but new requests should not be + // served + IndexInput indexInput = compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT); compositeDirectory.close(); - verify(localDirectory).close(); - verify(fileCache).remove(resolvedPath1); - verify(fileCache).remove(resolvedPath2); + // Any operations after close will throw AlreadyClosedException + assertThrows(AlreadyClosedException.class, () -> compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT)); + // Existing open IndexInputs will be served + indexInput.getFilePointer(); + indexInput.close(); + assertThrows(RuntimeException.class, indexInput::getFilePointer); + assertThrows(AlreadyClosedException.class, () -> compositeDirectory.close()); } public void testAfterSyncToRemote() throws IOException { - Path basePath = mock(Path.class); - Path resolvedPath = mock(Path.class); - when(basePath.resolve(anyString())).thenReturn(resolvedPath); - when(localDirectory.getDirectory()).thenReturn(basePath); - Collection files = Arrays.asList("_0.si", "_0.cfs"); - compositeDirectory.afterSyncToRemote(files); - verify(fileCache, times(files.size())).decRef(resolvedPath); + // File will be present locally until uploaded to Remote + assertTrue(existsInLocalDirectory("_1.cfe")); + compositeDirectory.afterSyncToRemote("_1.cfe"); + fileCache.prune(); + // After uploading to Remote, refCount will be decreased by 1 making it 0 and will be evicted if cache is pruned + assertFalse(existsInLocalDirectory("_1.cfe")); + } + + private void addFilesToDirectory(String[] files) throws IOException { + for (String file : files) { + IndexOutput indexOutput = compositeDirectory.createOutput(file, IOContext.DEFAULT); + indexOutput.close(); + } + } + + private void removeExtraFSFiles() throws IOException { + HashSet allFiles = new HashSet<>(Arrays.asList(localDirectory.listAll())); + allFiles.stream().filter(FileTypeUtils::isExtraFSFile).forEach(file -> { + try { + localDirectory.deleteFile(file); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + } + + private boolean existsInLocalDirectory(String name) throws IOException { + return Arrays.asList(localDirectory.listAll()).contains(name); + } + + private boolean existsInRemoteDirectory(String name) throws IOException { + return Arrays.asList(remoteSegmentStoreDirectory.listAll()).contains(name); + } + + private boolean existsInCompositeDirectory(String name) throws IOException { + return Arrays.asList(compositeDirectory.listAll()).contains(name); + } + + private Path getFilePath(String name) { + return localDirectory.getDirectory().resolve(name); } } diff --git a/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java index ee81369725e6f..3a1a149e76455 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java @@ -20,7 +20,9 @@ import org.opensearch.common.blobstore.BlobMetadata; import org.opensearch.common.blobstore.stream.write.WriteContext; import org.opensearch.common.blobstore.support.PlainBlobMetadata; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; import org.opensearch.core.action.ActionListener; +import org.opensearch.index.store.remote.utils.BlockIOContext; import org.opensearch.test.OpenSearchTestCase; import org.junit.Before; @@ -216,6 +218,14 @@ public void testOpenInput() throws IOException { assertTrue(indexInput instanceof RemoteIndexInput); assertEquals(100, indexInput.length()); verify(blobContainer).listBlobsByPrefixInSortedOrder("segment_1", 1, LEXICOGRAPHIC); + + BlockIOContext blockIOContext = new BlockIOContext(IOContext.DEFAULT, 100, 50); + when(blobContainer.readBlob("segment_1", 100, 50)).thenReturn(mockInputStream); + byte[] bytes = new byte[(int) blockIOContext.getBlockSize()]; + when(mockInputStream.readAllBytes()).thenReturn(bytes); + indexInput = remoteDirectory.openInput("segment_1", blockIOContext); + assertTrue(indexInput instanceof ByteArrayIndexInput); + assertEquals(blockIOContext.getBlockSize(), indexInput.length()); } public void testOpenInputWithLength() throws IOException { diff --git a/server/src/test/java/org/opensearch/index/store/remote/utils/cache/RefCountedCacheTestCase.java b/server/src/test/java/org/opensearch/index/store/remote/utils/cache/RefCountedCacheTestCase.java index b11740b53e11f..97e9fb288136d 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/utils/cache/RefCountedCacheTestCase.java +++ b/server/src/test/java/org/opensearch/index/store/remote/utils/cache/RefCountedCacheTestCase.java @@ -211,6 +211,15 @@ public void testComputeStats() { assertEquals(1, refCountedCache.stats().removeCount()); } + public void testClear() { + refCountedCache.put("1", 10L); + refCountedCache.put("2", 10L); + refCountedCache.put("3", 10L); + assertEquals(30L, refCountedCache.usage().usage()); + refCountedCache.clear(); + assertEquals(0L, refCountedCache.usage().usage()); + } + private void assertUsage(long usage, long activeUsage) { assertEquals(usage, refCountedCache.usage().usage()); assertEquals(activeUsage, refCountedCache.usage().activeUsage()); From f9d880bce260120014159752a5a1a81035c56d68 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Thu, 30 May 2024 14:03:55 +0530 Subject: [PATCH 16/22] Refactor usages of WRITEABLE_REMOTE_INDEX_SETTING to TIERED_REMOTE_INDEX_SETTING Signed-off-by: Shreyansh Ray --- .../java/org/opensearch/remotestore/WritableWarmIT.java | 2 +- .../cluster/metadata/MetadataCreateIndexService.java | 4 ++-- server/src/main/java/org/opensearch/index/IndexService.java | 2 +- server/src/main/java/org/opensearch/node/Node.java | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java index 690356bca24ce..2d4077d707b5e 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -59,7 +59,7 @@ protected boolean addMockIndexStorePlugin() { @Override protected Settings featureFlagSettings() { Settings.Builder featureSettings = Settings.builder(); - featureSettings.put(FeatureFlags.WRITEABLE_REMOTE_INDEX, true); + featureSettings.put(FeatureFlags.TIERED_REMOTE_INDEX, true); return featureSettings.build(); } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 014bd6b1885ea..6b5386fbb92e8 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1694,10 +1694,10 @@ static void validateTranslogDurabilitySettings(Settings requestSettings, Cluster public static void validateIndexStoreLocality(Settings indexSettings) { if (indexSettings.get(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.FULL.toString()) .equalsIgnoreCase(IndexModule.DataLocalityType.PARTIAL.toString()) - && !FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING)) { + && !FeatureFlags.isEnabled(FeatureFlags.TIERED_REMOTE_INDEX_SETTING)) { throw new IllegalArgumentException( "index.store.locality can be set to PARTIAL only if Feature Flag [" - + FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING.getKey() + + FeatureFlags.TIERED_REMOTE_INDEX_SETTING.getKey() + "] is set to true" ); } diff --git a/server/src/main/java/org/opensearch/index/IndexService.java b/server/src/main/java/org/opensearch/index/IndexService.java index 6cceae5ce7e9f..a7849bcf80474 100644 --- a/server/src/main/java/org/opensearch/index/IndexService.java +++ b/server/src/main/java/org/opensearch/index/IndexService.java @@ -612,7 +612,7 @@ public synchronized IndexShard createShard( } Directory directory = null; - if (FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING) && + if (FeatureFlags.isEnabled(FeatureFlags.TIERED_REMOTE_INDEX_SETTING) && // TODO : Need to remove this check after support for hot indices is added in Composite Directory this.indexSettings.isStoreLocalityPartial()) { Directory localDirectory = directoryFactory.newDirectory(this.indexSettings, path); diff --git a/server/src/main/java/org/opensearch/node/Node.java b/server/src/main/java/org/opensearch/node/Node.java index d42cb21be36dc..a91dce4ece126 100644 --- a/server/src/main/java/org/opensearch/node/Node.java +++ b/server/src/main/java/org/opensearch/node/Node.java @@ -2005,7 +2005,7 @@ DiscoveryNode getNode() { * Else it configures the size to 80% of available capacity for a dedicated search node, if not explicitly defined. */ private void initializeFileCache(Settings settings, CircuitBreaker circuitBreaker) throws IOException { - boolean isWritableRemoteIndexEnabled = FeatureFlags.isEnabled(FeatureFlags.WRITEABLE_REMOTE_INDEX_SETTING); + boolean isWritableRemoteIndexEnabled = FeatureFlags.isEnabled(FeatureFlags.TIERED_REMOTE_INDEX_SETTING); if (DiscoveryNode.isSearchNode(settings) || isWritableRemoteIndexEnabled) { NodeEnvironment.NodePath fileCacheNodePath = nodeEnvironment.fileCacheNodePath(); long capacity = NODE_SEARCH_CACHE_SIZE_SETTING.get(settings).getBytes(); From e1b18ad04c38b7fe6beda3fab6884c30d317b633 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Thu, 30 May 2024 16:00:06 +0530 Subject: [PATCH 17/22] Add tests for FileCachedIndexInput and review comment fixes Signed-off-by: Shreyansh Ray --- .../remotestore/WritableWarmIT.java | 3 +- .../index/store/RemoteDirectory.java | 22 +++-- .../filecache/FileCachedIndexInput.java | 14 +++ .../index/store/RemoteDirectoryTests.java | 7 +- .../filecache/FileCachedIndexInputTests.java | 91 +++++++++++++++++++ 5 files changed, 126 insertions(+), 11 deletions(-) create mode 100644 server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java index 2d4077d707b5e..e681c6a96cb03 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -28,7 +28,6 @@ import org.opensearch.indices.IndicesService; import org.opensearch.node.Node; import org.opensearch.test.OpenSearchIntegTestCase; -import org.opensearch.test.junit.annotations.TestLogging; import java.util.Arrays; import java.util.HashSet; @@ -41,7 +40,7 @@ @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) // Uncomment the below line to enable trace level logs for this test for better debugging -@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") +// @TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") public class WritableWarmIT extends RemoteStoreBaseIntegTestCase { protected static final String INDEX_NAME = "test-idx-1"; diff --git a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java index 1dfbb3dcc0f36..c61fae74c0bc0 100644 --- a/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/RemoteDirectory.java @@ -206,13 +206,7 @@ public IndexInput openInput(String name, long fileLength, IOContext context) thr InputStream inputStream = null; try { if (context instanceof BlockIOContext) { - long position = ((BlockIOContext) context).getBlockStart(); - long length = ((BlockIOContext) context).getBlockSize(); - inputStream = blobContainer.readBlob(name, position, length); - // TODO - Explore how we can buffer small chunks of data instead of having the whole 8MB block in memory - byte[] bytes = downloadRateLimiter.apply(inputStream).readAllBytes(); - inputStream.close(); - return new ByteArrayIndexInput(name, bytes); + return getBlockInput(name, fileLength, (BlockIOContext) context); } else { inputStream = blobContainer.readBlob(name); return new RemoteIndexInput(name, downloadRateLimiter.apply(inputStream), fileLength); @@ -446,4 +440,18 @@ private long calculateChecksumOfChecksum(Directory directory, String file) throw } } } + + private IndexInput getBlockInput(String name, long fileLength, BlockIOContext blockIOContext) throws IOException { + long position = blockIOContext.getBlockStart(); + long length = blockIOContext.getBlockSize(); + if (position < 0 || length < 0 || (position + length > fileLength)) { + throw new IllegalArgumentException("Invalid values of block start and size"); + } + byte[] bytes; + try (InputStream inputStream = blobContainer.readBlob(name, position, length)) { + // TODO - Explore how we can buffer small chunks of data instead of having the whole 8MB block in memory + bytes = downloadRateLimiter.apply(inputStream).readAllBytes(); + } + return new ByteArrayIndexInput(name, bytes); + } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java index 200a47e661ab4..7d915d4cdb080 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java @@ -133,6 +133,20 @@ public FileCachedIndexInput clone() { @Override public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { + if (offset < 0 || length < 0 || offset + length > this.length()) { + throw new IllegalArgumentException( + "slice() " + + sliceDescription + + " out of bounds: offset=" + + offset + + ",length=" + + length + + ",fileLength=" + + this.length() + + ": " + + this + ); + } IndexInput slicedIndexInput = luceneIndexInput.slice(sliceDescription, offset, length); cache.incRef(filePath); return new FileCachedIndexInput(cache, filePath, slicedIndexInput, true); diff --git a/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java index 3a1a149e76455..ed79a2b0bd8e4 100644 --- a/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/RemoteDirectoryTests.java @@ -219,8 +219,11 @@ public void testOpenInput() throws IOException { assertEquals(100, indexInput.length()); verify(blobContainer).listBlobsByPrefixInSortedOrder("segment_1", 1, LEXICOGRAPHIC); - BlockIOContext blockIOContext = new BlockIOContext(IOContext.DEFAULT, 100, 50); - when(blobContainer.readBlob("segment_1", 100, 50)).thenReturn(mockInputStream); + BlockIOContext blockIOContextInvalidValues = new BlockIOContext(IOContext.DEFAULT, 10, 1000); + assertThrows(IllegalArgumentException.class, () -> remoteDirectory.openInput("segment_1", blockIOContextInvalidValues)); + + BlockIOContext blockIOContext = new BlockIOContext(IOContext.DEFAULT, 10, 50); + when(blobContainer.readBlob("segment_1", 10, 50)).thenReturn(mockInputStream); byte[] bytes = new byte[(int) blockIOContext.getBlockSize()]; when(mockInputStream.readAllBytes()).thenReturn(bytes); indexInput = remoteDirectory.openInput("segment_1", blockIOContext); diff --git a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java new file mode 100644 index 0000000000000..5282dbcd3c465 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java @@ -0,0 +1,91 @@ +/* + * 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.store.remote.filecache; + +import org.apache.lucene.store.FSDirectory; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; +import org.opensearch.core.common.breaker.CircuitBreaker; +import org.opensearch.core.common.breaker.NoopCircuitBreaker; +import org.opensearch.test.OpenSearchTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.nio.file.Path; + +public class FileCachedIndexInputTests extends OpenSearchTestCase { + + private FileCache fileCache; + private Path filePath; + private IndexInput underlyingIndexInput; + private FileCachedIndexInput fileCachedIndexInput; + + private static final int FILE_CACHE_CAPACITY = 1000; + private static final String TEST_FILE = "test_file"; + private static final String SLICE_DESC = "slice_description"; + + @Before + public void setup() throws IOException { + Path basePath = createTempDir("FileCachedIndexInputTests"); + FSDirectory fsDirectory = FSDirectory.open(basePath); + IndexOutput indexOutput = fsDirectory.createOutput(TEST_FILE, IOContext.DEFAULT); + // Writing to the file so that it's size is not zero + indexOutput.writeInt(100); + indexOutput.close(); + filePath = basePath.resolve(TEST_FILE); + underlyingIndexInput = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); + fileCache = FileCacheFactory.createConcurrentLRUFileCache(FILE_CACHE_CAPACITY, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + fileCachedIndexInput = new FileCachedIndexInput(fileCache, filePath, underlyingIndexInput); + fileCache.put(filePath, new FullFileCachedIndexInput(fileCache, filePath, fileCachedIndexInput)); + } + + public void testClone() throws IOException { + + // Since the file ia already in cache and has refCount 1, activeUsage and totalUsage will be same + assertTrue(isActiveAndTotalUsageSame()); + + // Decrementing the refCount explicitly on the file which will make it inactive (as refCount will drop to 0) + fileCache.decRef(filePath); + assertFalse(isActiveAndTotalUsageSame()); + + // After cloning the refCount will increase again and activeUsage and totalUsage will be same again + FileCachedIndexInput clonedFileCachedIndexInput = fileCachedIndexInput.clone(); + assertTrue(isActiveAndTotalUsageSame()); + + // Closing the clone will again decrease the refCount making it 0 + clonedFileCachedIndexInput.close(); + assertFalse(isActiveAndTotalUsageSame()); + } + + public void testSlice() throws IOException { + + // Throw IllegalArgumentException if offset is negative + assertThrows(IllegalArgumentException.class, () -> fileCachedIndexInput.slice(SLICE_DESC, -1, 10)); + + // Throw IllegalArgumentException if length is negative + assertThrows(IllegalArgumentException.class, () -> fileCachedIndexInput.slice(SLICE_DESC, 5, -1)); + + // Decrementing the refCount explicitly on the file which will make it inactive (as refCount will drop to 0) + fileCache.decRef(filePath); + assertFalse(isActiveAndTotalUsageSame()); + + // Creating a slice will increase the refCount + IndexInput slicedFileCachedIndexInput = fileCachedIndexInput.slice(SLICE_DESC, 1, 2); + assertTrue(isActiveAndTotalUsageSame()); + + // Closing the clone will again decrease the refCount making it 0 + slicedFileCachedIndexInput.close(); + assertFalse(isActiveAndTotalUsageSame()); + } + + private boolean isActiveAndTotalUsageSame() { + return fileCache.usage().activeUsage() == fileCache.usage().usage(); + } +} From 5e008db0fccd8fd1e23500134f58396ae87008a6 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Thu, 30 May 2024 16:55:58 +0530 Subject: [PATCH 18/22] Add additional IT for feature flag disabled Signed-off-by: Shreyansh Ray --- .../remotestore/WritableWarmIT.java | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java index e681c6a96cb03..8ff737041d7ef 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -59,10 +59,28 @@ protected boolean addMockIndexStorePlugin() { protected Settings featureFlagSettings() { Settings.Builder featureSettings = Settings.builder(); featureSettings.put(FeatureFlags.TIERED_REMOTE_INDEX, true); - return featureSettings.build(); } + public void testWritableWarmFeatureFlagDisabled() { + Settings clusterSettings = Settings.builder().put(super.nodeSettings(0)).put(FeatureFlags.TIERED_REMOTE_INDEX, false).build(); + internalCluster().startDataOnlyNodes(1, clusterSettings); + + Settings indexSettings = Settings.builder() + .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) + .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.PARTIAL.name()) + .build(); + + assertThrows( + "index.store.locality can be set to PARTIAL only if Feature Flag [" + + FeatureFlags.TIERED_REMOTE_INDEX_SETTING.getKey() + + "] is set to true", + IllegalArgumentException.class, + () -> client().admin().indices().prepareCreate(INDEX_NAME).setSettings(indexSettings).get() + ); + } + public void testWritableWarmBasic() throws Exception { Settings settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) From c1e6b181098f14eabaf0114531b9bbce33b3cc83 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Wed, 12 Jun 2024 19:08:38 +0530 Subject: [PATCH 19/22] Move setting for Partial Locality type behind Feature Flag, fix bug for ref count via cloneMap in FullFileCachedIndexInput and other review fixes Signed-off-by: Shreyansh Ray --- .../remotestore/WritableWarmIT.java | 53 +++++---- .../metadata/MetadataCreateIndexService.java | 13 --- .../common/settings/IndexScopedSettings.java | 7 +- .../shard/RemoteStoreRefreshListener.java | 5 - .../store/CloseableFilterIndexOutput.java | 8 +- .../index/store/CompositeDirectory.java | 31 ++---- .../filecache/FileCachedIndexInput.java | 23 +--- .../filecache/FullFileCachedIndexInput.java | 101 ++++++++++-------- .../FullFileCachedIndexInputImpl.java | 74 +++++++++++++ .../store/remote/utils/FileTypeUtils.java | 4 + .../index/store/CompositeDirectoryTests.java | 27 +---- .../filecache/FileCachedIndexInputTests.java | 40 +++---- .../FullFileCachedIndexInputTests.java | 68 ++++++++++++ ...ansferManagerBlobContainerReaderTests.java | 43 ++++++++ ...sferManagerRemoteDirectoryReaderTests.java | 51 +++++++++ ...ests.java => TransferManagerTestCase.java} | 37 +++---- 16 files changed, 383 insertions(+), 202 deletions(-) create mode 100644 server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputImpl.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerBlobContainerReaderTests.java create mode 100644 server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerRemoteDirectoryReaderTests.java rename server/src/test/java/org/opensearch/index/store/remote/utils/{TransferManagerTests.java => TransferManagerTestCase.java} (87%) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java index 8ff737041d7ef..3e74e2c8de074 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -12,11 +12,13 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.FilterDirectory; +import org.opensearch.action.admin.indices.delete.DeleteIndexRequest; import org.opensearch.action.admin.indices.get.GetIndexRequest; import org.opensearch.action.admin.indices.get.GetIndexResponse; import org.opensearch.action.search.SearchResponse; import org.opensearch.cluster.metadata.IndexMetadata; import org.opensearch.common.settings.Settings; +import org.opensearch.common.settings.SettingsException; import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.query.QueryBuilders; @@ -24,10 +26,11 @@ import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.utils.FileTypeUtils; -import org.opensearch.index.store.remote.utils.cache.CacheUsage; import org.opensearch.indices.IndicesService; import org.opensearch.node.Node; +import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; +import org.opensearch.test.junit.annotations.TestLogging; import java.util.Arrays; import java.util.HashSet; @@ -38,9 +41,9 @@ import static org.opensearch.test.hamcrest.OpenSearchAssertions.assertHitCount; @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) -@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 1, numClientNodes = 0, supportsDedicatedMasters = false) +@OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0, supportsDedicatedMasters = false) // Uncomment the below line to enable trace level logs for this test for better debugging -// @TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") +@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") public class WritableWarmIT extends RemoteStoreBaseIntegTestCase { protected static final String INDEX_NAME = "test-idx-1"; @@ -64,7 +67,9 @@ protected Settings featureFlagSettings() { public void testWritableWarmFeatureFlagDisabled() { Settings clusterSettings = Settings.builder().put(super.nodeSettings(0)).put(FeatureFlags.TIERED_REMOTE_INDEX, false).build(); - internalCluster().startDataOnlyNodes(1, clusterSettings); + InternalTestCluster internalTestCluster = internalCluster(); + internalTestCluster.startClusterManagerOnlyNodes(3, clusterSettings); + internalTestCluster.startDataOnlyNodes(1, clusterSettings); Settings indexSettings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) @@ -72,16 +77,24 @@ public void testWritableWarmFeatureFlagDisabled() { .put(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.PARTIAL.name()) .build(); - assertThrows( - "index.store.locality can be set to PARTIAL only if Feature Flag [" - + FeatureFlags.TIERED_REMOTE_INDEX_SETTING.getKey() - + "] is set to true", - IllegalArgumentException.class, - () -> client().admin().indices().prepareCreate(INDEX_NAME).setSettings(indexSettings).get() - ); + try { + prepareCreate(INDEX_NAME).setSettings(indexSettings).get(); + fail("Should have thrown Exception as setting should not be registered if Feature Flag is Disabled"); + } catch (SettingsException ex) { + assertEquals( + "unknown setting [" + + IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey() + + "] please check that any required plugins are installed, or check the " + + "breaking changes documentation for removed settings", + ex.getMessage() + ); + } } public void testWritableWarmBasic() throws Exception { + InternalTestCluster internalTestCluster = internalCluster(); + internalTestCluster.startClusterManagerOnlyNode(); + internalTestCluster.startDataOnlyNode(); Settings settings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) .put(IndexMetadata.SETTING_NUMBER_OF_REPLICAS, 0) @@ -112,8 +125,8 @@ public void testWritableWarmBasic() throws Exception { indexBulk(INDEX_NAME, NUM_DOCS_IN_BULK); flushAndRefresh(INDEX_NAME); - FileCache fileCache = internalCluster().getDataNodeInstance(Node.class).fileCache(); - IndexShard shard = internalCluster().getDataNodeInstance(IndicesService.class) + FileCache fileCache = internalTestCluster.getDataNodeInstance(Node.class).fileCache(); + IndexShard shard = internalTestCluster.getDataNodeInstance(IndicesService.class) .indexService(resolveIndex(INDEX_NAME)) .getShardOrNull(0); Directory directory = (((FilterDirectory) (((FilterDirectory) (shard.store().directory())).getDelegate())).getDelegate()); @@ -124,22 +137,18 @@ public void testWritableWarmBasic() throws Exception { flushAndRefresh(INDEX_NAME); Set filesAfterMerge = new HashSet<>(Arrays.asList(directory.listAll())); - CacheUsage usageBeforePrune = fileCache.usage(); - fileCache.prune(); - CacheUsage usageAfterPrune = fileCache.usage(); - Set filesFromPreviousGenStillPresent = filesBeforeMerge.stream() .filter(filesAfterMerge::contains) .filter(file -> !FileTypeUtils.isLockFile(file)) + .filter(file -> !FileTypeUtils.isSegmentsFile(file)) .collect(Collectors.toUnmodifiableSet()); // Asserting that after merge all the files from previous gen are no more part of the directory assertTrue(filesFromPreviousGenStillPresent.isEmpty()); - // Asserting that after the merge, refCount of some files in FileCache dropped to zero which resulted in their eviction after - // pruning - assertTrue(usageAfterPrune.usage() < usageBeforePrune.usage()); - // Clearing the file cache to avoid any file leaks - fileCache.clear(); + // Deleting the index (so that ref count drops to zero for all the files) and then pruning the cache to clear it to avoid any file + // leaks + assertAcked(client().admin().indices().delete(new DeleteIndexRequest(INDEX_NAME)).get()); + fileCache.prune(); } } diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 6b5386fbb92e8..0ba5d48c58247 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -75,7 +75,6 @@ import org.opensearch.common.settings.Setting; import org.opensearch.common.settings.Settings; import org.opensearch.common.unit.TimeValue; -import org.opensearch.common.util.FeatureFlags; import org.opensearch.common.xcontent.XContentHelper; import org.opensearch.core.action.ActionListener; import org.opensearch.core.common.Strings; @@ -996,7 +995,6 @@ static Settings aggregateIndexSettings( validateStoreTypeSettings(indexSettings); validateRefreshIntervalSettings(request.settings(), clusterSettings); validateTranslogDurabilitySettings(request.settings(), clusterSettings, settings); - validateIndexStoreLocality(request.settings()); return indexSettings; } @@ -1691,15 +1689,4 @@ static void validateTranslogDurabilitySettings(Settings requestSettings, Cluster } - public static void validateIndexStoreLocality(Settings indexSettings) { - if (indexSettings.get(IndexModule.INDEX_STORE_LOCALITY_SETTING.getKey(), IndexModule.DataLocalityType.FULL.toString()) - .equalsIgnoreCase(IndexModule.DataLocalityType.PARTIAL.toString()) - && !FeatureFlags.isEnabled(FeatureFlags.TIERED_REMOTE_INDEX_SETTING)) { - throw new IllegalArgumentException( - "index.store.locality can be set to PARTIAL only if Feature Flag [" - + FeatureFlags.TIERED_REMOTE_INDEX_SETTING.getKey() - + "] is set to true" - ); - } - } } diff --git a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java index 19b3b13dfd396..1488f5d30b4ba 100644 --- a/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/opensearch/common/settings/IndexScopedSettings.java @@ -41,6 +41,7 @@ import org.opensearch.common.annotation.PublicApi; import org.opensearch.common.logging.Loggers; import org.opensearch.common.settings.Setting.Property; +import org.opensearch.common.util.FeatureFlags; import org.opensearch.index.IndexModule; import org.opensearch.index.IndexSettings; import org.opensearch.index.IndexSortConfig; @@ -188,7 +189,6 @@ public final class IndexScopedSettings extends AbstractScopedSettings { MapperService.INDEX_MAPPING_FIELD_NAME_LENGTH_LIMIT_SETTING, BitsetFilterCache.INDEX_LOAD_RANDOM_ACCESS_FILTERS_EAGERLY_SETTING, IndexModule.INDEX_STORE_TYPE_SETTING, - IndexModule.INDEX_STORE_LOCALITY_SETTING, IndexModule.INDEX_STORE_PRE_LOAD_SETTING, IndexModule.INDEX_STORE_HYBRID_MMAP_EXTENSIONS, IndexModule.INDEX_STORE_HYBRID_NIO_EXTENSIONS, @@ -261,7 +261,10 @@ public final class IndexScopedSettings extends AbstractScopedSettings { * is ready for production release, the feature flag can be removed, and the * setting should be moved to {@link #BUILT_IN_INDEX_SETTINGS}. */ - public static final Map> FEATURE_FLAGGED_INDEX_SETTINGS = Map.of(); + public static final Map> FEATURE_FLAGGED_INDEX_SETTINGS = Map.of( + FeatureFlags.TIERED_REMOTE_INDEX, + List.of(IndexModule.INDEX_STORE_LOCALITY_SETTING) + ); public static final IndexScopedSettings DEFAULT_SCOPED_SETTINGS = new IndexScopedSettings(Settings.EMPTY, BUILT_IN_INDEX_SETTINGS); 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 601fb56876039..3db5e0b7f4b0c 100644 --- a/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java +++ b/server/src/main/java/org/opensearch/index/shard/RemoteStoreRefreshListener.java @@ -262,11 +262,6 @@ private boolean syncSegments() { Map localSegmentsSizeMap = updateLocalSizeMapAndTracker(localSegmentsPostRefresh).entrySet() .stream() .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - Collection segmentsToRefresh = localSegmentsPostRefresh.stream() - .filter(file -> !skipUpload(file)) - .collect(Collectors.toList()); - CountDownLatch latch = new CountDownLatch(1); ActionListener segmentUploadsCompletedListener = new LatchedActionListener<>(new ActionListener<>() { @Override diff --git a/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java index 3a4309fe6ee6d..a3cedcb84d400 100644 --- a/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java +++ b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java @@ -13,6 +13,7 @@ import org.opensearch.common.lucene.store.FilterIndexOutput; import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; /** * FilterIndexOutput which takes in an additional FunctionalInterface as a parameter to perform required operations once the IndexOutput is closed @@ -32,16 +33,21 @@ public interface OnCloseListener { private final OnCloseListener onCloseListener; private final String fileName; + private final AtomicBoolean isClosed; public CloseableFilterIndexOutput(IndexOutput out, String fileName, OnCloseListener onCloseListener) { super("CloseableFilterIndexOutput for file " + fileName, out); this.fileName = fileName; this.onCloseListener = onCloseListener; + this.isClosed = new AtomicBoolean(false); } @Override public void close() throws IOException { super.close(); - onCloseListener.onClose(fileName); + if (isClosed.get() == false) { + onCloseListener.onClose(fileName); + isClosed.set(true); + } } } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index ecc6a95475170..57f1cef314b2d 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -23,7 +23,7 @@ import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.filecache.FullFileCachedIndexInput; +import org.opensearch.index.store.remote.filecache.FullFileCachedIndexInputImpl; import org.opensearch.index.store.remote.utils.BlockIOContext; import org.opensearch.index.store.remote.utils.FileTypeUtils; import org.opensearch.index.store.remote.utils.TransferManager; @@ -53,7 +53,6 @@ public class CompositeDirectory extends FilterDirectory { private final RemoteSegmentStoreDirectory remoteDirectory; private final FileCache fileCache; private final TransferManager transferManager; - private final Set pendingDeletes; /** * Constructor to initialise the composite directory @@ -67,7 +66,6 @@ public CompositeDirectory(Directory localDirectory, Directory remoteDirectory, F this.localDirectory = (FSDirectory) localDirectory; this.remoteDirectory = (RemoteSegmentStoreDirectory) remoteDirectory; this.fileCache = fileCache; - this.pendingDeletes = new HashSet<>(); transferManager = new TransferManager( (name, position, length) -> new InputStreamIndexInput( remoteDirectory.openInput(name, new BlockIOContext(IOContext.DEFAULT, position, length)), @@ -95,7 +93,6 @@ public String[] listAll() throws IOException { logger.trace("Composite Directory[{}]: Remote Directory files - {}", this::toString, () -> Arrays.toString(remoteFiles)); Set nonBlockLuceneFiles = allFiles.stream() .filter(file -> !FileTypeUtils.isBlockFile(file)) - .filter(file -> !pendingDeletes.contains(file)) .collect(Collectors.toUnmodifiableSet()); String[] files = new String[nonBlockLuceneFiles.size()]; nonBlockLuceneFiles.toArray(files); @@ -113,14 +110,13 @@ public String[] listAll() throws IOException { @Override public void deleteFile(String name) throws IOException { ensureOpen(); - ensureFileNotDeleted(name); logger.trace("Composite Directory[{}]: deleteFile() called {}", this::toString, () -> name); if (FileTypeUtils.isTempFile(name)) { localDirectory.deleteFile(name); } else if (Arrays.asList(listAll()).contains(name) == false) { throw new NoSuchFileException("File " + name + " not found in directory"); } else { - pendingDeletes.add(name); + fileCache.remove(localDirectory.getDirectory().resolve(name)); } } @@ -133,7 +129,6 @@ public void deleteFile(String name) throws IOException { @Override public long fileLength(String name) throws IOException { ensureOpen(); - ensureFileNotDeleted(name); logger.trace("Composite Directory[{}]: fileLength() called {}", this::toString, () -> name); long fileLength; Path key = localDirectory.getDirectory().resolve(name); @@ -170,10 +165,6 @@ public long fileLength(String name) throws IOException { @Override public IndexOutput createOutput(String name, IOContext context) throws IOException { ensureOpen(); - // If file was deleted earlier, removing it from the deleted list - if (pendingDeletes.contains(name)) { - pendingDeletes.remove(name); - } logger.trace("Composite Directory[{}]: createOutput() called {}", this::toString, () -> name); // The CloseableFilterIndexOutput will ensure that the file is added to FileCache once write is completed on this file return new CloseableFilterIndexOutput(localDirectory.createOutput(name, context), name, this::cacheFile); @@ -201,7 +192,6 @@ public void sync(Collection names) throws IOException { @Override public void rename(String source, String dest) throws IOException { ensureOpen(); - ensureFileNotDeleted(source); logger.trace("Composite Directory[{}]: rename() called : source-{}, dest-{}", this::toString, () -> source, () -> dest); localDirectory.rename(source, dest); fileCache.remove(localDirectory.getDirectory().resolve(source)); @@ -217,7 +207,6 @@ public void rename(String source, String dest) throws IOException { @Override public IndexInput openInput(String name, IOContext context) throws IOException { ensureOpen(); - ensureFileNotDeleted(name); logger.trace("Composite Directory[{}]: openInput() called {}", this::toString, () -> name); // We aren't tracking temporary files (created via createTempOutput) currently in FileCache as these are created and then deleted // within a very short span of time @@ -269,6 +258,11 @@ public void close() throws IOException { localDirectory.close(); } + @Override + public String toString() { + return "Composite Directory @ " + Integer.toHexString(hashCode()); + } + /** * Function to perform operations once files have been uploaded to Remote Store * Currently deleting the local files here, as once uploaded to Remote, local files become eligible for eviction from FileCache @@ -314,15 +308,6 @@ private void validate(Directory localDirectory, Directory remoteDirectory, FileC ); } - /** - * Ensure that the file has not already been deleted - */ - private void ensureFileNotDeleted(String name) throws IOException { - if (pendingDeletes.contains(name)) { - throw new NoSuchFileException("File " + name + " is already pending delete"); - } - } - /** * Return the list of files present in Remote */ @@ -348,7 +333,7 @@ private void cacheFile(String name) throws IOException { // this is just a temporary solution, will pin the file once support for that is added in FileCache // TODO : Pin the above filePath in the file cache once pinning support is added so that it cannot be evicted unless it has been // successfully uploaded to Remote - fileCache.put(filePath, new FullFileCachedIndexInput(fileCache, filePath, localDirectory.openInput(name, IOContext.READ))); + fileCache.put(filePath, new FullFileCachedIndexInputImpl(fileCache, filePath, localDirectory.openInput(name, IOContext.DEFAULT))); } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java index 7d915d4cdb080..ab6f5f931da0f 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInput.java @@ -39,9 +39,9 @@ public class FileCachedIndexInput extends IndexInput implements RandomAccessInpu protected IndexInput luceneIndexInput; /** indicates if this IndexInput instance is a clone or not */ - private final boolean isClone; + protected final boolean isClone; - private volatile boolean closed = false; + protected volatile boolean closed = false; public FileCachedIndexInput(FileCache cache, Path filePath, IndexInput underlyingIndexInput) { this(cache, filePath, underlyingIndexInput, false); @@ -133,23 +133,8 @@ public FileCachedIndexInput clone() { @Override public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { - if (offset < 0 || length < 0 || offset + length > this.length()) { - throw new IllegalArgumentException( - "slice() " - + sliceDescription - + " out of bounds: offset=" - + offset - + ",length=" - + length - + ",fileLength=" - + this.length() - + ": " - + this - ); - } - IndexInput slicedIndexInput = luceneIndexInput.slice(sliceDescription, offset, length); - cache.incRef(filePath); - return new FileCachedIndexInput(cache, filePath, slicedIndexInput, true); + // never reach here! + throw new UnsupportedOperationException("FileCachedIndexInput couldn't be sliced."); } @Override diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java index f8aed0432cba8..a70efe353a38d 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java @@ -8,67 +8,80 @@ package org.opensearch.index.store.remote.filecache; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IndexInput; -import org.opensearch.common.annotation.ExperimentalApi; +import java.io.IOException; import java.nio.file.Path; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.HashSet; +import java.util.Set; -/** - * Implementation of the CachedIndexInput for NON_BLOCK files which takes in an IndexInput as parameter - * - * @opensearch.experimental - */ -@ExperimentalApi -public class FullFileCachedIndexInput implements CachedIndexInput { - private final FileCache fileCache; - private final Path path; - private final FileCachedIndexInput fileCachedIndexInput; - private final AtomicBoolean isClosed; +public class FullFileCachedIndexInput extends FileCachedIndexInput { + private static final Logger logger = LogManager.getLogger(FullFileCachedIndexInput.class); + private final Set clones; - /** - * Constructor - takes IndexInput as parameter - */ - public FullFileCachedIndexInput(FileCache fileCache, Path path, IndexInput indexInput) { - this.fileCache = fileCache; - this.path = path; - fileCachedIndexInput = new FileCachedIndexInput(fileCache, path, indexInput); - isClosed = new AtomicBoolean(false); + public FullFileCachedIndexInput(FileCache cache, Path filePath, IndexInput underlyingIndexInput) { + this(cache, filePath, underlyingIndexInput, false); } - /** - * Returns the wrapped indexInput - */ - @Override - public IndexInput getIndexInput() { - if (isClosed.get()) throw new AlreadyClosedException("Index input is already closed"); - return fileCachedIndexInput; + public FullFileCachedIndexInput(FileCache cache, Path filePath, IndexInput underlyingIndexInput, boolean isClone) { + super(cache, filePath, underlyingIndexInput, isClone); + clones = new HashSet<>(); } - /** - * Returns the length of the wrapped indexInput - */ @Override - public long length() { - return fileCachedIndexInput.length(); + public FullFileCachedIndexInput clone() { + FullFileCachedIndexInput clonedIndexInput = new FullFileCachedIndexInput(cache, filePath, luceneIndexInput.clone(), true); + cache.incRef(filePath); + clones.add(clonedIndexInput); + return clonedIndexInput; } - /** - * Checks if the wrapped indexInput is closed - */ @Override - public boolean isClosed() { - return isClosed.get(); + public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { + if (offset < 0 || length < 0 || offset + length > this.length()) { + throw new IllegalArgumentException( + "slice() " + + sliceDescription + + " out of bounds: offset=" + + offset + + ",length=" + + length + + ",fileLength=" + + this.length() + + ": " + + this + ); + } + IndexInput slicedLuceneIndexInput = luceneIndexInput.slice(sliceDescription, offset, length); + FullFileCachedIndexInput slicedIndexInput = new FullFileCachedIndexInput(cache, filePath, slicedLuceneIndexInput, true); + clones.add(slicedIndexInput); + cache.incRef(filePath); + return slicedIndexInput; } - /** - * Closes the wrapped indexInput - */ @Override - public void close() throws Exception { - if (!isClosed.getAndSet(true)) { - fileCachedIndexInput.close(); + public void close() throws IOException { + if (!closed) { + if (isClone) { + cache.decRef(filePath); + } + clones.forEach(indexInput -> { + try { + indexInput.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + try { + luceneIndexInput.close(); + } catch (AlreadyClosedException e) { + logger.trace("FullFileCachedIndexInput already closed"); + } + luceneIndexInput = null; + closed = true; } } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputImpl.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputImpl.java new file mode 100644 index 0000000000000..f756877aa5a9f --- /dev/null +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputImpl.java @@ -0,0 +1,74 @@ +/* + * 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.store.remote.filecache; + +import org.apache.lucene.store.AlreadyClosedException; +import org.apache.lucene.store.IndexInput; +import org.opensearch.common.annotation.ExperimentalApi; + +import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * Implementation of the CachedIndexInput for NON_BLOCK files which takes in an IndexInput as parameter + * + * @opensearch.experimental + */ +@ExperimentalApi +public class FullFileCachedIndexInputImpl implements CachedIndexInput { + private final FileCache fileCache; + private final Path path; + private final FullFileCachedIndexInput fullFileCachedIndexInput; + private final AtomicBoolean isClosed; + + /** + * Constructor - takes IndexInput as parameter + */ + public FullFileCachedIndexInputImpl(FileCache fileCache, Path path, IndexInput indexInput) { + this.fileCache = fileCache; + this.path = path; + fullFileCachedIndexInput = new FullFileCachedIndexInput(fileCache, path, indexInput); + isClosed = new AtomicBoolean(false); + } + + /** + * Returns the wrapped indexInput + */ + @Override + public IndexInput getIndexInput() { + if (isClosed.get()) throw new AlreadyClosedException("Index input is already closed"); + return fullFileCachedIndexInput; + } + + /** + * Returns the length of the wrapped indexInput + */ + @Override + public long length() { + return fullFileCachedIndexInput.length(); + } + + /** + * Checks if the wrapped indexInput is closed + */ + @Override + public boolean isClosed() { + return isClosed.get(); + } + + /** + * Closes the wrapped indexInput + */ + @Override + public void close() throws Exception { + if (!isClosed.getAndSet(true)) { + fullFileCachedIndexInput.close(); + } + } +} diff --git a/server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java b/server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java index c20533e0e5ff1..e78480bd500ee 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java +++ b/server/src/main/java/org/opensearch/index/store/remote/utils/FileTypeUtils.java @@ -33,4 +33,8 @@ public static boolean isExtraFSFile(String name) { public static boolean isLockFile(String name) { return name.endsWith(".lock"); } + + public static boolean isSegmentsFile(String name) { + return name.startsWith("segments_"); + } } diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java index 2045172f623bf..4bacf8cbaf842 100644 --- a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -67,34 +67,11 @@ public void testListAll() throws IOException { public void testDeleteFile() throws IOException { assertTrue(existsInCompositeDirectory(FILE_PRESENT_LOCALLY)); - // Read existing file from directory - IndexInput indexInput = compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT); // Delete the file and assert that it no more is a part of the directory compositeDirectory.deleteFile(FILE_PRESENT_LOCALLY); assertFalse(existsInCompositeDirectory(FILE_PRESENT_LOCALLY)); - // On trying any operations(such as openInput) on the deleted file again, it should throw NoSuchFileException - assertThrows( - "File _1.cfe is already pending delete", - NoSuchFileException.class, - () -> compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT) - ); - // Trying to delete it again should also result in NoSuchFileException - assertThrows( - "File" + FILE_PRESENT_LOCALLY + " is already pending delete", - NoSuchFileException.class, - () -> compositeDirectory.deleteFile("_1.cfe") - ); - // Even though the file is deleted, IndexInputs opened prior to deletion should work fine until closed - indexInput.getFilePointer(); - indexInput.close(); - assertThrows(RuntimeException.class, indexInput::getFilePointer); - // Files which do not exist in directory should throw NoSuchFileException - assertFalse(existsInCompositeDirectory(NON_EXISTENT_FILE)); - assertThrows( - "File " + NON_EXISTENT_FILE + " not found in directory", - NoSuchFileException.class, - () -> compositeDirectory.deleteFile(NON_EXISTENT_FILE) - ); + // Reading deleted file from directory should result in NoSuchFileException + assertThrows(NoSuchFileException.class, () -> compositeDirectory.openInput(FILE_PRESENT_LOCALLY, IOContext.DEFAULT)); } public void testFileLength() throws IOException { diff --git a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java index 5282dbcd3c465..8f71737bf1568 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java @@ -22,14 +22,14 @@ public class FileCachedIndexInputTests extends OpenSearchTestCase { - private FileCache fileCache; - private Path filePath; - private IndexInput underlyingIndexInput; + protected FileCache fileCache; + protected Path filePath; + protected IndexInput underlyingIndexInput; private FileCachedIndexInput fileCachedIndexInput; - private static final int FILE_CACHE_CAPACITY = 1000; - private static final String TEST_FILE = "test_file"; - private static final String SLICE_DESC = "slice_description"; + protected static final int FILE_CACHE_CAPACITY = 1000; + protected static final String TEST_FILE = "test_file"; + protected static final String SLICE_DESC = "slice_description"; @Before public void setup() throws IOException { @@ -42,11 +42,15 @@ public void setup() throws IOException { filePath = basePath.resolve(TEST_FILE); underlyingIndexInput = fsDirectory.openInput(TEST_FILE, IOContext.DEFAULT); fileCache = FileCacheFactory.createConcurrentLRUFileCache(FILE_CACHE_CAPACITY, new NoopCircuitBreaker(CircuitBreaker.REQUEST)); + } + + protected void setupIndexInputAndAddToFileCache() { fileCachedIndexInput = new FileCachedIndexInput(fileCache, filePath, underlyingIndexInput); - fileCache.put(filePath, new FullFileCachedIndexInput(fileCache, filePath, fileCachedIndexInput)); + fileCache.put(filePath, new FullFileCachedIndexInputImpl(fileCache, filePath, fileCachedIndexInput)); } public void testClone() throws IOException { + setupIndexInputAndAddToFileCache(); // Since the file ia already in cache and has refCount 1, activeUsage and totalUsage will be same assertTrue(isActiveAndTotalUsageSame()); @@ -65,27 +69,11 @@ public void testClone() throws IOException { } public void testSlice() throws IOException { - - // Throw IllegalArgumentException if offset is negative - assertThrows(IllegalArgumentException.class, () -> fileCachedIndexInput.slice(SLICE_DESC, -1, 10)); - - // Throw IllegalArgumentException if length is negative - assertThrows(IllegalArgumentException.class, () -> fileCachedIndexInput.slice(SLICE_DESC, 5, -1)); - - // Decrementing the refCount explicitly on the file which will make it inactive (as refCount will drop to 0) - fileCache.decRef(filePath); - assertFalse(isActiveAndTotalUsageSame()); - - // Creating a slice will increase the refCount - IndexInput slicedFileCachedIndexInput = fileCachedIndexInput.slice(SLICE_DESC, 1, 2); - assertTrue(isActiveAndTotalUsageSame()); - - // Closing the clone will again decrease the refCount making it 0 - slicedFileCachedIndexInput.close(); - assertFalse(isActiveAndTotalUsageSame()); + setupIndexInputAndAddToFileCache(); + assertThrows(UnsupportedOperationException.class, () -> fileCachedIndexInput.slice(SLICE_DESC, 10, 100)); } - private boolean isActiveAndTotalUsageSame() { + protected boolean isActiveAndTotalUsageSame() { return fileCache.usage().activeUsage() == fileCache.usage().usage(); } } diff --git a/server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java b/server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java new file mode 100644 index 0000000000000..504ad81e1b458 --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java @@ -0,0 +1,68 @@ +/* + * 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.store.remote.filecache; + +import org.apache.lucene.store.IndexInput; + +import java.io.IOException; + +public class FullFileCachedIndexInputTests extends FileCachedIndexInputTests { + private FullFileCachedIndexInput fullFileCachedIndexInput; + + @Override + protected void setupIndexInputAndAddToFileCache() { + fullFileCachedIndexInput = new FullFileCachedIndexInput(fileCache, filePath, underlyingIndexInput); + fileCache.put(filePath, new FullFileCachedIndexInputImpl(fileCache, filePath, fullFileCachedIndexInput)); + } + + @Override + public void testClone() throws IOException { + setupIndexInputAndAddToFileCache(); + + // Since the file ia already in cache and has refCount 1, activeUsage and totalUsage will be same + assertTrue(isActiveAndTotalUsageSame()); + + // Decrementing the refCount explicitly on the file which will make it inactive (as refCount will drop to 0) + fileCache.decRef(filePath); + assertFalse(isActiveAndTotalUsageSame()); + + // After cloning the refCount will increase again and activeUsage and totalUsage will be same again + FileCachedIndexInput clonedFileCachedIndexInput1 = fullFileCachedIndexInput.clone(); + FileCachedIndexInput clonedFileCachedIndexInput2 = clonedFileCachedIndexInput1.clone(); + FileCachedIndexInput clonedFileCachedIndexInput3 = clonedFileCachedIndexInput2.clone(); + assertTrue(isActiveAndTotalUsageSame()); + + // Closing the parent will close all the clones decreasing the refCount to 0 + fullFileCachedIndexInput.close(); + assertFalse(isActiveAndTotalUsageSame()); + } + + @Override + public void testSlice() throws IOException { + setupIndexInputAndAddToFileCache(); + + // Throw IllegalArgumentException if offset is negative + assertThrows(IllegalArgumentException.class, () -> fullFileCachedIndexInput.slice(SLICE_DESC, -1, 10)); + + // Throw IllegalArgumentException if length is negative + assertThrows(IllegalArgumentException.class, () -> fullFileCachedIndexInput.slice(SLICE_DESC, 5, -1)); + + // Decrementing the refCount explicitly on the file which will make it inactive (as refCount will drop to 0) + fileCache.decRef(filePath); + assertFalse(isActiveAndTotalUsageSame()); + + // Creating a slice will increase the refCount + IndexInput slicedFileCachedIndexInput = fullFileCachedIndexInput.slice(SLICE_DESC, 1, 2); + assertTrue(isActiveAndTotalUsageSame()); + + // Closing the parent will close all the slices as well decreasing the refCount to 0 + fullFileCachedIndexInput.close(); + assertFalse(isActiveAndTotalUsageSame()); + } +} diff --git a/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerBlobContainerReaderTests.java b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerBlobContainerReaderTests.java new file mode 100644 index 0000000000000..24e57cde3ffda --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerBlobContainerReaderTests.java @@ -0,0 +1,43 @@ +/* + * 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.store.remote.utils; + +import org.opensearch.common.blobstore.BlobContainer; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.concurrent.CountDownLatch; + +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; + +public class TransferManagerBlobContainerReaderTests extends TransferManagerTestCase { + private BlobContainer blobContainer; + + @Override + protected void initializeTransferManager() throws IOException { + blobContainer = mock(BlobContainer.class); + doAnswer(i -> new ByteArrayInputStream(createData())).when(blobContainer).readBlob(eq("blob"), anyLong(), anyLong()); + transferManager = new TransferManager(blobContainer::readBlob, fileCache); + } + + protected void mockExceptionWhileReading() throws IOException { + doThrow(new IOException("Expected test exception")).when(blobContainer).readBlob(eq("failure-blob"), anyLong(), anyLong()); + } + + protected void mockWaitForLatchReader(CountDownLatch latch) throws IOException { + doAnswer(i -> { + latch.await(); + return new ByteArrayInputStream(createData()); + }).when(blobContainer).readBlob(eq("blocking-blob"), anyLong(), anyLong()); + } +} diff --git a/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerRemoteDirectoryReaderTests.java b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerRemoteDirectoryReaderTests.java new file mode 100644 index 0000000000000..e777a287bf10f --- /dev/null +++ b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerRemoteDirectoryReaderTests.java @@ -0,0 +1,51 @@ +/* + * 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.store.remote.utils; + +import org.apache.lucene.store.IOContext; +import org.opensearch.common.lucene.store.ByteArrayIndexInput; +import org.opensearch.common.lucene.store.InputStreamIndexInput; +import org.opensearch.index.store.RemoteDirectory; + +import java.io.IOException; +import java.util.concurrent.CountDownLatch; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; + +public class TransferManagerRemoteDirectoryReaderTests extends TransferManagerTestCase { + private RemoteDirectory remoteDirectory; + + @Override + protected void initializeTransferManager() throws IOException { + remoteDirectory = mock(RemoteDirectory.class); + doAnswer(i -> new ByteArrayIndexInput("blob", createData())).when(remoteDirectory).openInput(eq("blob"), any()); + transferManager = new TransferManager( + (name, position, length) -> new InputStreamIndexInput( + remoteDirectory.openInput(name, new BlockIOContext(IOContext.DEFAULT, position, length)), + length + ), + fileCache + ); + } + + protected void mockExceptionWhileReading() throws IOException { + doThrow(new IOException("Expected test exception")).when(remoteDirectory).openInput(eq("failure-blob"), any()); + } + + protected void mockWaitForLatchReader(CountDownLatch latch) throws IOException { + doAnswer(i -> { + latch.await(); + return new ByteArrayIndexInput("blocking-blob", createData()); + }).when(remoteDirectory).openInput(eq("blocking-blob"), any()); + } +} diff --git a/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTests.java b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTestCase.java similarity index 87% rename from server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTests.java rename to server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTestCase.java index c0a5ea749b765..810a4c336fdf7 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/utils/TransferManagerTestCase.java @@ -13,7 +13,6 @@ import org.apache.lucene.store.IndexInput; import org.apache.lucene.store.MMapDirectory; import org.apache.lucene.store.SimpleFSLockFactory; -import org.opensearch.common.blobstore.BlobContainer; import org.opensearch.core.common.breaker.CircuitBreaker; import org.opensearch.core.common.breaker.NoopCircuitBreaker; import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; @@ -24,7 +23,6 @@ import org.junit.After; import org.junit.Before; -import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -36,31 +34,23 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; -import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.Mockito.doAnswer; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.mock; @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) -public class TransferManagerTests extends OpenSearchTestCase { - private static final int EIGHT_MB = 1024 * 1024 * 8; - private final FileCache fileCache = FileCacheFactory.createConcurrentLRUFileCache( +public abstract class TransferManagerTestCase extends OpenSearchTestCase { + protected static final int EIGHT_MB = 1024 * 1024 * 8; + protected final FileCache fileCache = FileCacheFactory.createConcurrentLRUFileCache( EIGHT_MB * 2, 1, new NoopCircuitBreaker(CircuitBreaker.REQUEST) ); - private MMapDirectory directory; - private BlobContainer blobContainer; - private TransferManager transferManager; + protected MMapDirectory directory; + protected TransferManager transferManager; @Before public void setUp() throws Exception { super.setUp(); directory = new MMapDirectory(createTempDir(), SimpleFSLockFactory.INSTANCE); - blobContainer = mock(BlobContainer.class); - doAnswer(i -> new ByteArrayInputStream(createData())).when(blobContainer).readBlob(eq("blob"), anyLong(), anyLong()); - transferManager = new TransferManager(blobContainer::readBlob, fileCache); + initializeTransferManager(); } @After @@ -68,7 +58,7 @@ public void tearDown() throws Exception { super.tearDown(); } - private static byte[] createData() { + protected static byte[] createData() { final byte[] data = new byte[EIGHT_MB]; data[EIGHT_MB - 1] = 7; return data; @@ -162,7 +152,7 @@ public void testUsageExceedsCapacity() throws Exception { } public void testDownloadFails() throws Exception { - doThrow(new IOException("Expected test exception")).when(blobContainer).readBlob(eq("failure-blob"), anyLong(), anyLong()); + mockExceptionWhileReading(); List blobParts = new ArrayList<>(); blobParts.add(new BlobFetchRequest.BlobPart("failure-blob", 0, EIGHT_MB)); expectThrows( @@ -177,10 +167,7 @@ public void testFetchesToDifferentBlobsDoNotBlockOnEachOther() throws Exception // Mock a call for a blob that will block until the latch is released, // then start the fetch for that blob on a separate thread final CountDownLatch latch = new CountDownLatch(1); - doAnswer(i -> { - latch.await(); - return new ByteArrayInputStream(createData()); - }).when(blobContainer).readBlob(eq("blocking-blob"), anyLong(), anyLong()); + mockWaitForLatchReader(latch); List blobParts = new ArrayList<>(); blobParts.add(new BlobFetchRequest.BlobPart("blocking-blob", 0, EIGHT_MB)); @@ -206,6 +193,12 @@ public void testFetchesToDifferentBlobsDoNotBlockOnEachOther() throws Exception assertFalse(blockingThread.isAlive()); } + protected abstract void initializeTransferManager() throws IOException; + + protected abstract void mockExceptionWhileReading() throws IOException; + + protected abstract void mockWaitForLatchReader(CountDownLatch latch) throws IOException; + private IndexInput fetchBlobWithName(String blobname) throws IOException { List blobParts = new ArrayList<>(); blobParts.add(new BlobFetchRequest.BlobPart("blob", 0, EIGHT_MB)); From 74f9c29a5bc8583c61965581e3979e3b545b7faa Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Fri, 14 Jun 2024 19:33:40 +0530 Subject: [PATCH 20/22] Minor test and nit fixes Signed-off-by: Shreyansh Ray --- .../remotestore/WritableWarmIT.java | 7 ++++++ .../metadata/MetadataCreateIndexService.java | 1 - .../store/CloseableFilterIndexOutput.java | 2 +- .../index/store/CompositeDirectory.java | 23 +++++++++++-------- ...mpl.java => CachedFullFileIndexInput.java} | 4 ++-- .../filecache/FullFileCachedIndexInput.java | 5 ++-- .../index/store/CompositeDirectoryTests.java | 8 ++++--- .../filecache/FileCachedIndexInputTests.java | 2 +- .../FullFileCachedIndexInputTests.java | 19 +++++++++++---- 9 files changed, 48 insertions(+), 23 deletions(-) rename server/src/main/java/org/opensearch/index/store/remote/filecache/{FullFileCachedIndexInputImpl.java => CachedFullFileIndexInput.java} (91%) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java index 3e74e2c8de074..d6b18ce2297df 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -23,6 +23,7 @@ import org.opensearch.index.IndexModule; import org.opensearch.index.query.QueryBuilders; import org.opensearch.index.shard.IndexShard; +import org.opensearch.index.store.CompositeDirectory; import org.opensearch.index.store.remote.file.CleanerDaemonThreadLeakFilter; import org.opensearch.index.store.remote.filecache.FileCache; import org.opensearch.index.store.remote.utils.FileTypeUtils; @@ -146,6 +147,12 @@ public void testWritableWarmBasic() throws Exception { // Asserting that after merge all the files from previous gen are no more part of the directory assertTrue(filesFromPreviousGenStillPresent.isEmpty()); + // Asserting that files from previous gen are not present in File Cache as well + filesBeforeMerge.stream() + .filter(file -> !FileTypeUtils.isLockFile(file)) + .filter(file -> !FileTypeUtils.isSegmentsFile(file)) + .forEach(file -> assertNull(fileCache.get(((CompositeDirectory) directory).getFilePath(file)))); + // Deleting the index (so that ref count drops to zero for all the files) and then pruning the cache to clear it to avoid any file // leaks assertAcked(client().admin().indices().delete(new DeleteIndexRequest(INDEX_NAME)).get()); diff --git a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java index 0ba5d48c58247..16edec112f123 100644 --- a/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java +++ b/server/src/main/java/org/opensearch/cluster/metadata/MetadataCreateIndexService.java @@ -1688,5 +1688,4 @@ static void validateTranslogDurabilitySettings(Settings requestSettings, Cluster } } - } diff --git a/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java index a3cedcb84d400..291f714369a74 100644 --- a/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java +++ b/server/src/main/java/org/opensearch/index/store/CloseableFilterIndexOutput.java @@ -44,8 +44,8 @@ public CloseableFilterIndexOutput(IndexOutput out, String fileName, OnCloseListe @Override public void close() throws IOException { - super.close(); if (isClosed.get() == false) { + super.close(); onCloseListener.onClose(fileName); isClosed.set(true); } diff --git a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java index 57f1cef314b2d..eb89c86ae687f 100644 --- a/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java +++ b/server/src/main/java/org/opensearch/index/store/CompositeDirectory.java @@ -21,9 +21,9 @@ import org.opensearch.common.lucene.store.InputStreamIndexInput; import org.opensearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; import org.opensearch.index.store.remote.file.OnDemandBlockSnapshotIndexInput; +import org.opensearch.index.store.remote.filecache.CachedFullFileIndexInput; import org.opensearch.index.store.remote.filecache.CachedIndexInput; import org.opensearch.index.store.remote.filecache.FileCache; -import org.opensearch.index.store.remote.filecache.FullFileCachedIndexInputImpl; import org.opensearch.index.store.remote.utils.BlockIOContext; import org.opensearch.index.store.remote.utils.FileTypeUtils; import org.opensearch.index.store.remote.utils.TransferManager; @@ -116,7 +116,7 @@ public void deleteFile(String name) throws IOException { } else if (Arrays.asList(listAll()).contains(name) == false) { throw new NoSuchFileException("File " + name + " not found in directory"); } else { - fileCache.remove(localDirectory.getDirectory().resolve(name)); + fileCache.remove(getFilePath(name)); } } @@ -131,7 +131,7 @@ public long fileLength(String name) throws IOException { ensureOpen(); logger.trace("Composite Directory[{}]: fileLength() called {}", this::toString, () -> name); long fileLength; - Path key = localDirectory.getDirectory().resolve(name); + Path key = getFilePath(name); if (FileTypeUtils.isTempFile(name) || fileCache.get(key) != null) { try { fileLength = localDirectory.fileLength(name); @@ -194,7 +194,7 @@ public void rename(String source, String dest) throws IOException { ensureOpen(); logger.trace("Composite Directory[{}]: rename() called : source-{}, dest-{}", this::toString, () -> source, () -> dest); localDirectory.rename(source, dest); - fileCache.remove(localDirectory.getDirectory().resolve(source)); + fileCache.remove(getFilePath(source)); cacheFile(dest); } @@ -215,7 +215,7 @@ public IndexInput openInput(String name, IOContext context) throws IOException { return localDirectory.openInput(name, context); } // Return directly from the FileCache (via TransferManager) if complete file is present - Path key = localDirectory.getDirectory().resolve(name); + Path key = getFilePath(name); CachedIndexInput indexInput = fileCache.get(key); if (indexInput != null) { logger.trace("Composite Directory[{}]: Complete file {} found in FileCache", this::toString, () -> name); @@ -281,8 +281,13 @@ Uncomment the below commented line(to remove the file from cache once uploaded) this::toString, () -> file ); - fileCache.decRef(localDirectory.getDirectory().resolve(file)); - // fileCache.remove(localDirectory.getDirectory().resolve(fileName)); + fileCache.decRef(getFilePath(file)); + // fileCache.remove(getFilePath(fileName)); + } + + // Visibility public since we need it in IT tests + public Path getFilePath(String name) { + return localDirectory.getDirectory().resolve(name); } /** @@ -327,13 +332,13 @@ private String[] getRemoteFiles() throws IOException { } private void cacheFile(String name) throws IOException { - Path filePath = localDirectory.getDirectory().resolve(name); + Path filePath = getFilePath(name); // put will increase the refCount for the path, making sure it is not evicted, will decrease the ref after it is uploaded to Remote // so that it can be evicted after that // this is just a temporary solution, will pin the file once support for that is added in FileCache // TODO : Pin the above filePath in the file cache once pinning support is added so that it cannot be evicted unless it has been // successfully uploaded to Remote - fileCache.put(filePath, new FullFileCachedIndexInputImpl(fileCache, filePath, localDirectory.openInput(name, IOContext.DEFAULT))); + fileCache.put(filePath, new CachedFullFileIndexInput(fileCache, filePath, localDirectory.openInput(name, IOContext.DEFAULT))); } } diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputImpl.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java similarity index 91% rename from server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputImpl.java rename to server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java index f756877aa5a9f..c159e3ac61465 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputImpl.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java @@ -21,7 +21,7 @@ * @opensearch.experimental */ @ExperimentalApi -public class FullFileCachedIndexInputImpl implements CachedIndexInput { +public class CachedFullFileIndexInput implements CachedIndexInput { private final FileCache fileCache; private final Path path; private final FullFileCachedIndexInput fullFileCachedIndexInput; @@ -30,7 +30,7 @@ public class FullFileCachedIndexInputImpl implements CachedIndexInput { /** * Constructor - takes IndexInput as parameter */ - public FullFileCachedIndexInputImpl(FileCache fileCache, Path path, IndexInput indexInput) { + public CachedFullFileIndexInput(FileCache fileCache, Path path, IndexInput indexInput) { this.fileCache = fileCache; this.path = path; fullFileCachedIndexInput = new FullFileCachedIndexInput(fileCache, path, indexInput); diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java index a70efe353a38d..7411c14d59ef4 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java @@ -71,8 +71,8 @@ public void close() throws IOException { clones.forEach(indexInput -> { try { indexInput.close(); - } catch (IOException e) { - throw new RuntimeException(e); + } catch (Exception e) { + logger.trace("Exception while closing clone - {}", e.getMessage()); } }); try { @@ -81,6 +81,7 @@ public void close() throws IOException { logger.trace("FullFileCachedIndexInput already closed"); } luceneIndexInput = null; + clones.clear(); closed = true; } } diff --git a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java index 4bacf8cbaf842..d5628cfab9ee7 100644 --- a/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java +++ b/server/src/test/java/org/opensearch/index/store/CompositeDirectoryTests.java @@ -157,11 +157,13 @@ public void testClose() throws IOException { public void testAfterSyncToRemote() throws IOException { // File will be present locally until uploaded to Remote - assertTrue(existsInLocalDirectory("_1.cfe")); - compositeDirectory.afterSyncToRemote("_1.cfe"); + assertTrue(existsInLocalDirectory(FILE_PRESENT_LOCALLY)); + compositeDirectory.afterSyncToRemote(FILE_PRESENT_LOCALLY); fileCache.prune(); // After uploading to Remote, refCount will be decreased by 1 making it 0 and will be evicted if cache is pruned - assertFalse(existsInLocalDirectory("_1.cfe")); + assertFalse(existsInLocalDirectory(FILE_PRESENT_LOCALLY)); + // Asserting file is not present in FileCache + assertNull(fileCache.get(getFilePath(FILE_PRESENT_LOCALLY))); } private void addFilesToDirectory(String[] files) throws IOException { diff --git a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java index 8f71737bf1568..258bc2db4c5d0 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/filecache/FileCachedIndexInputTests.java @@ -46,7 +46,7 @@ public void setup() throws IOException { protected void setupIndexInputAndAddToFileCache() { fileCachedIndexInput = new FileCachedIndexInput(fileCache, filePath, underlyingIndexInput); - fileCache.put(filePath, new FullFileCachedIndexInputImpl(fileCache, filePath, fileCachedIndexInput)); + fileCache.put(filePath, new CachedFullFileIndexInput(fileCache, filePath, fileCachedIndexInput)); } public void testClone() throws IOException { diff --git a/server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java b/server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java index 504ad81e1b458..7fb7a03584e20 100644 --- a/server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java +++ b/server/src/test/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInputTests.java @@ -8,6 +8,7 @@ package org.opensearch.index.store.remote.filecache; +import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IndexInput; import java.io.IOException; @@ -18,16 +19,20 @@ public class FullFileCachedIndexInputTests extends FileCachedIndexInputTests { @Override protected void setupIndexInputAndAddToFileCache() { fullFileCachedIndexInput = new FullFileCachedIndexInput(fileCache, filePath, underlyingIndexInput); - fileCache.put(filePath, new FullFileCachedIndexInputImpl(fileCache, filePath, fullFileCachedIndexInput)); + fileCache.put(filePath, new CachedFullFileIndexInput(fileCache, filePath, fullFileCachedIndexInput)); } @Override public void testClone() throws IOException { setupIndexInputAndAddToFileCache(); - // Since the file ia already in cache and has refCount 1, activeUsage and totalUsage will be same + // Since the file is already in cache and has refCount 1, activeUsage and totalUsage will be same assertTrue(isActiveAndTotalUsageSame()); + // Getting the file cache entry (which wil increase the ref count, hence doing dec ref immediately afterwards) + CachedIndexInput cachedIndexInput = fileCache.get(filePath); + fileCache.decRef(filePath); + // Decrementing the refCount explicitly on the file which will make it inactive (as refCount will drop to 0) fileCache.decRef(filePath); assertFalse(isActiveAndTotalUsageSame()); @@ -38,9 +43,15 @@ public void testClone() throws IOException { FileCachedIndexInput clonedFileCachedIndexInput3 = clonedFileCachedIndexInput2.clone(); assertTrue(isActiveAndTotalUsageSame()); - // Closing the parent will close all the clones decreasing the refCount to 0 - fullFileCachedIndexInput.close(); + // closing the first level clone will close all subsequent level clones and reduce ref count to 0 + clonedFileCachedIndexInput1.close(); assertFalse(isActiveAndTotalUsageSame()); + + fileCache.prune(); + + // since the file cache entry was evicted the corresponding CachedIndexInput will be closed and will throw exception when trying to + // read the index input + assertThrows(AlreadyClosedException.class, cachedIndexInput::getIndexInput); } @Override From 7fce3d3e1b13efad392ed035b167c2fe75600f8d Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Mon, 17 Jun 2024 14:42:36 +0530 Subject: [PATCH 21/22] Add javadocs for FullFileCachedIndexInput Signed-off-by: Shreyansh Ray --- .../remotestore/WritableWarmIT.java | 7 +++---- .../filecache/CachedFullFileIndexInput.java | 2 +- .../filecache/FullFileCachedIndexInput.java | 21 +++++++++++++++++++ 3 files changed, 25 insertions(+), 5 deletions(-) diff --git a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java index d6b18ce2297df..a51bd6b20fff0 100644 --- a/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java +++ b/server/src/internalClusterTest/java/org/opensearch/remotestore/WritableWarmIT.java @@ -31,7 +31,6 @@ import org.opensearch.node.Node; import org.opensearch.test.InternalTestCluster; import org.opensearch.test.OpenSearchIntegTestCase; -import org.opensearch.test.junit.annotations.TestLogging; import java.util.Arrays; import java.util.HashSet; @@ -44,7 +43,7 @@ @ThreadLeakFilters(filters = CleanerDaemonThreadLeakFilter.class) @OpenSearchIntegTestCase.ClusterScope(scope = OpenSearchIntegTestCase.Scope.TEST, numDataNodes = 0, supportsDedicatedMasters = false) // Uncomment the below line to enable trace level logs for this test for better debugging -@TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") +// @TestLogging(reason = "Getting trace logs from composite directory package", value = "org.opensearch.index.store:TRACE") public class WritableWarmIT extends RemoteStoreBaseIntegTestCase { protected static final String INDEX_NAME = "test-idx-1"; @@ -69,8 +68,8 @@ protected Settings featureFlagSettings() { public void testWritableWarmFeatureFlagDisabled() { Settings clusterSettings = Settings.builder().put(super.nodeSettings(0)).put(FeatureFlags.TIERED_REMOTE_INDEX, false).build(); InternalTestCluster internalTestCluster = internalCluster(); - internalTestCluster.startClusterManagerOnlyNodes(3, clusterSettings); - internalTestCluster.startDataOnlyNodes(1, clusterSettings); + internalTestCluster.startClusterManagerOnlyNode(clusterSettings); + internalTestCluster.startDataOnlyNode(clusterSettings); Settings indexSettings = Settings.builder() .put(IndexMetadata.SETTING_NUMBER_OF_SHARDS, 1) diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java index c159e3ac61465..286739cb6cd90 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/CachedFullFileIndexInput.java @@ -16,7 +16,7 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * Implementation of the CachedIndexInput for NON_BLOCK files which takes in an IndexInput as parameter + * Implementation of the CachedIndexInput for full files which takes in an IndexInput as parameter * * @opensearch.experimental */ diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java index 7411c14d59ef4..269a3f0579455 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java @@ -12,12 +12,21 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.IndexInput; +import org.opensearch.common.annotation.ExperimentalApi; import java.io.IOException; import java.nio.file.Path; import java.util.HashSet; import java.util.Set; +/** + * Extension of {@link FileCachedIndexInput} for full files for handling clones and slices + * We maintain a clone map so that we can close them when the parent IndexInput is closed so that ref count is properly maintained in file cache + * Closing of clones explicitly is needed as Lucene does not guarantee that it will close the clones + * https://github.com/apache/lucene/blob/8340b01c3cc229f33584ce2178b07b8984daa6a9/lucene/core/src/java/org/apache/lucene/store/IndexInput.java#L32-L33 + * @opensearch.experimental + */ +@ExperimentalApi public class FullFileCachedIndexInput extends FileCachedIndexInput { private static final Logger logger = LogManager.getLogger(FullFileCachedIndexInput.class); private final Set clones; @@ -31,6 +40,10 @@ public FullFileCachedIndexInput(FileCache cache, Path filePath, IndexInput under clones = new HashSet<>(); } + /** + * Clones the index input and returns the clone + * Increase the ref count whenever the index input is cloned and add it to the clone map as well + */ @Override public FullFileCachedIndexInput clone() { FullFileCachedIndexInput clonedIndexInput = new FullFileCachedIndexInput(cache, filePath, luceneIndexInput.clone(), true); @@ -39,6 +52,10 @@ public FullFileCachedIndexInput clone() { return clonedIndexInput; } + /** + * Clones the index input and returns the slice + * Increase the ref count whenever the index input is sliced and add it to the clone map as well + */ @Override public IndexInput slice(String sliceDescription, long offset, long length) throws IOException { if (offset < 0 || length < 0 || offset + length > this.length()) { @@ -62,6 +79,10 @@ public IndexInput slice(String sliceDescription, long offset, long length) throw return slicedIndexInput; } + /** + * Closes the index input and it's clones as well + * @throws IOException + */ @Override public void close() throws IOException { if (!closed) { From 901849b17a3b60a9f9ca690664780b77fbc64399 Mon Sep 17 00:00:00 2001 From: Shreyansh Ray Date: Mon, 17 Jun 2024 15:07:50 +0530 Subject: [PATCH 22/22] Minor precommit fixes Signed-off-by: Shreyansh Ray --- .../index/store/remote/filecache/FullFileCachedIndexInput.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java index 269a3f0579455..9383c53d6d830 100644 --- a/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java +++ b/server/src/main/java/org/opensearch/index/store/remote/filecache/FullFileCachedIndexInput.java @@ -81,7 +81,6 @@ public IndexInput slice(String sliceDescription, long offset, long length) throw /** * Closes the index input and it's clones as well - * @throws IOException */ @Override public void close() throws IOException {